You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/13 09:06:23 UTC

[GitHub] [flink] dianfu opened a new pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

dianfu opened a new pull request #13611:
URL: https://github.com/apache/flink/pull/13611


   
   ## What is the purpose of the change
   
   *This pull request add support of RichFunction in Python DataStream API*
   
   
   ## Verifying this change
   
   
   This change is already covered by existing tests, such as test_datastream.py.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] shuiqiangchen commented on a change in pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
shuiqiangchen commented on a change in pull request #13611:
URL: https://github.com/apache/flink/pull/13611#discussion_r504657426



##########
File path: flink-python/pyflink/fn_execution/beam/beam_operations_fast.pyx
##########
@@ -185,23 +187,36 @@ cdef class DataStreamStatelessFunctionOperation(BeamStatelessFunctionOperation):
         super(DataStreamStatelessFunctionOperation, self).__init__(name, spec, counter_factory,
                                                                    sampler, consumers)
 
-    def generate_func(self, udfs):
-        func = operation_utils.extract_data_stream_stateless_funcs(udfs)
-        return func, []
+    def open_func(self):
+        for user_defined_func in self.user_defined_funcs:
+            runtime_context = RuntimeContext(

Review comment:
       Can we reuse the runtime_context by creating it outside of the for loop?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * d0e3faf04d6f9e24387d07b8175ff33d205426e6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569) 
   * a6c9e93ce58216aef76c4e6709e62db740cf616b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586) 
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] shuiqiangchen commented on a change in pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
shuiqiangchen commented on a change in pull request #13611:
URL: https://github.com/apache/flink/pull/13611#discussion_r504755235



##########
File path: flink-python/pyflink/datastream/data_stream.py
##########
@@ -467,25 +466,26 @@ def partition_custom(self, partitioner: Union[Callable, Partitioner],
             raise TypeError("Parameter partitioner should be a type of Partitioner.")
 
         gateway = get_gateway()
-        data_stream_num_partitions_env_key = \
-            gateway.jvm.PythonPartitionCustomOperator.DATA_STREAM_NUM_PARTITIONS
 
         class PartitionCustomMapFunction(MapFunction):
             """
             A wrapper class for partition_custom map function. It indicates that it is a partition
-            custom operation that we need to apply DataStreamPythonPartitionCustomFunctionOperator
+            custom operation that we need to apply PythonPartitionCustomOperator
             to run the map function.
             """
 
             def __init__(self):
                 self.num_partitions = None
 
-            def map(self, value):
-                return self.partition_custom_map(value)
+            def open(self, runtime_context: RuntimeContext):
+                self.num_partitions = int(runtime_context.get_job_parameter(
+                    "DATA_STREAM_NUM_PARTITIONS", "-1"))
+                if self.num_partitions <= 0:
+                    raise ValueError(
+                        "The partition number should be a positive value, got %s"
+                        % self.num_partitions)

Review comment:
       It would be better to do parameter validation in PythonPartitionCustomOperator when setting the numPartitions before executing the job.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533) 
   * bbf13d4f68c7c6450d264d862eaf9db45ff80e6d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * a6c9e93ce58216aef76c4e6709e62db740cf616b Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586) 
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 UNKNOWN
   * 6c395eda1cf7170a0996921ce039457e95928867 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * bbf13d4f68c7c6450d264d862eaf9db45ff80e6d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558) 
   * d0e3faf04d6f9e24387d07b8175ff33d205426e6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] shuiqiangchen commented on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
shuiqiangchen commented on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-708838133


   @dianfu Thank you for you clarification. The PR looks good to me now. +1


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d9856d22f60e139eea7a8333039915c50b209129",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613",
       "triggerID" : "d9856d22f60e139eea7a8333039915c50b209129",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 6c395eda1cf7170a0996921ce039457e95928867 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590) 
   * d9856d22f60e139eea7a8333039915c50b209129 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] shuiqiangchen commented on a change in pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
shuiqiangchen commented on a change in pull request #13611:
URL: https://github.com/apache/flink/pull/13611#discussion_r504656094



##########
File path: flink-python/pyflink/fn_execution/beam/beam_operations_slow.py
##########
@@ -162,23 +165,36 @@ def __init__(self, name, spec, counter_factory, sampler, consumers):
         super(DataStreamStatelessFunctionOperation, self).__init__(name, spec, counter_factory,
                                                                    sampler, consumers)
 
-    def generate_func(self, udfs):
-        func = operation_utils.extract_data_stream_stateless_funcs(udfs=udfs)
-        return lambda it: map(func, it), []
+    def open_func(self):
+        for user_defined_func in self.user_defined_funcs:
+            runtime_context = RuntimeContext(

Review comment:
       Can we reuse the runtime_context by creating it outside of the for loop?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512) 
   * 48d8c611eb953320c664fdf5d6c5bd1610350bc9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525) 
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dianfu closed pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
dianfu closed pull request #13611:
URL: https://github.com/apache/flink/pull/13611


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dianfu commented on a change in pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
dianfu commented on a change in pull request #13611:
URL: https://github.com/apache/flink/pull/13611#discussion_r505107742



##########
File path: flink-python/pyflink/datastream/data_stream.py
##########
@@ -467,25 +466,26 @@ def partition_custom(self, partitioner: Union[Callable, Partitioner],
             raise TypeError("Parameter partitioner should be a type of Partitioner.")
 
         gateway = get_gateway()
-        data_stream_num_partitions_env_key = \
-            gateway.jvm.PythonPartitionCustomOperator.DATA_STREAM_NUM_PARTITIONS
 
         class PartitionCustomMapFunction(MapFunction):
             """
             A wrapper class for partition_custom map function. It indicates that it is a partition
-            custom operation that we need to apply DataStreamPythonPartitionCustomFunctionOperator
+            custom operation that we need to apply PythonPartitionCustomOperator
             to run the map function.
             """
 
             def __init__(self):
                 self.num_partitions = None
 
-            def map(self, value):
-                return self.partition_custom_map(value)
+            def open(self, runtime_context: RuntimeContext):
+                self.num_partitions = int(runtime_context.get_job_parameter(
+                    "DATA_STREAM_NUM_PARTITIONS", "-1"))
+                if self.num_partitions <= 0:
+                    raise ValueError(
+                        "The partition number should be a positive value, got %s"
+                        % self.num_partitions)

Review comment:
       In PythonPartitionCustomOperator, the numPartitions will always be positive as it's the parallelism. So the validation there is not necessary. The validation here checks that the numPartitions is transmitted to the Python user-defined function correctly. Otherwise, it will be -1. Thoughts?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * bbf13d4f68c7c6450d264d862eaf9db45ff80e6d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558) 
   * d0e3faf04d6f9e24387d07b8175ff33d205426e6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dianfu commented on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
dianfu commented on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-708838823


   @shuiqiangchen Thanks a lot for the review. Not sure why the tests are not triggered. The tests have passed in my personal azure pipeline: https://dev.azure.com/dianfu/Flink/_build/results?buildId=172&view=results. Merging...


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * d0e3faf04d6f9e24387d07b8175ff33d205426e6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569) 
   * a6c9e93ce58216aef76c4e6709e62db740cf616b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512) 
   * 48d8c611eb953320c664fdf5d6c5bd1610350bc9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525) 
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d9856d22f60e139eea7a8333039915c50b209129",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d9856d22f60e139eea7a8333039915c50b209129",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595) 
   * 6c395eda1cf7170a0996921ce039457e95928867 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590) 
   * d9856d22f60e139eea7a8333039915c50b209129 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d9856d22f60e139eea7a8333039915c50b209129",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d9856d22f60e139eea7a8333039915c50b209129",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * a6c9e93ce58216aef76c4e6709e62db740cf616b Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586) 
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595) 
   * 6c395eda1cf7170a0996921ce039457e95928867 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590) 
   * d9856d22f60e139eea7a8333039915c50b209129 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d9856d22f60e139eea7a8333039915c50b209129",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613",
       "triggerID" : "d9856d22f60e139eea7a8333039915c50b209129",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * d9856d22f60e139eea7a8333039915c50b209129 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d9856d22f60e139eea7a8333039915c50b209129",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613",
       "triggerID" : "d9856d22f60e139eea7a8333039915c50b209129",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595) 
   * 6c395eda1cf7170a0996921ce039457e95928867 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590) 
   * d9856d22f60e139eea7a8333039915c50b209129 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7613) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48d8c611eb953320c664fdf5d6c5bd1610350bc9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525) 
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595",
       "triggerID" : "1e9048f293577c1e2b11e85f0bafd5575e73cc04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6c395eda1cf7170a0996921ce039457e95928867",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590",
       "triggerID" : "6c395eda1cf7170a0996921ce039457e95928867",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * a6c9e93ce58216aef76c4e6709e62db740cf616b Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7586) 
   * 1e9048f293577c1e2b11e85f0bafd5575e73cc04 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7595) 
   * 6c395eda1cf7170a0996921ce039457e95928867 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7590) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569",
       "triggerID" : "d0e3faf04d6f9e24387d07b8175ff33d205426e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a6c9e93ce58216aef76c4e6709e62db740cf616b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * d0e3faf04d6f9e24387d07b8175ff33d205426e6 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7569) 
   * a6c9e93ce58216aef76c4e6709e62db740cf616b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512) 
   * 48d8c611eb953320c664fdf5d6c5bd1610350bc9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * bbf13d4f68c7c6450d264d862eaf9db45ff80e6d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707604758


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit eb4e4e077d403d0b04703da38ed7c605ab237a81 (Tue Oct 13 09:08:24 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558",
       "triggerID" : "bbf13d4f68c7c6450d264d862eaf9db45ff80e6d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7533) 
   * bbf13d4f68c7c6450d264d862eaf9db45ff80e6d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7558) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13611:
URL: https://github.com/apache/flink/pull/13611#issuecomment-707634048


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512",
       "triggerID" : "eb4e4e077d403d0b04703da38ed7c605ab237a81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525",
       "triggerID" : "48d8c611eb953320c664fdf5d6c5bd1610350bc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "51d4364176c11ed1b766a00bdbab9597f7ca4af2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb4e4e077d403d0b04703da38ed7c605ab237a81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7512) 
   * 48d8c611eb953320c664fdf5d6c5bd1610350bc9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7525) 
   * 97bf0d8b9d1541d8ecb077fa977e2fe3c7a15d2d UNKNOWN
   * 51d4364176c11ed1b766a00bdbab9597f7ca4af2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dianfu commented on a change in pull request #13611: [FLINK-19480][python] Support RichFunction in Python DataStream API

Posted by GitBox <gi...@apache.org>.
dianfu commented on a change in pull request #13611:
URL: https://github.com/apache/flink/pull/13611#discussion_r505105967



##########
File path: flink-python/pyflink/fn_execution/beam/beam_operations_slow.py
##########
@@ -162,23 +165,36 @@ def __init__(self, name, spec, counter_factory, sampler, consumers):
         super(DataStreamStatelessFunctionOperation, self).__init__(name, spec, counter_factory,
                                                                    sampler, consumers)
 
-    def generate_func(self, udfs):
-        func = operation_utils.extract_data_stream_stateless_funcs(udfs=udfs)
-        return lambda it: map(func, it), []
+    def open_func(self):
+        for user_defined_func in self.user_defined_funcs:
+            runtime_context = RuntimeContext(

Review comment:
       Actually there is only one user-defined function for Python DataStream API. Besides, I think that the runtime context may not be shared even if there are multiple user-defined functions, e.g. when we support metrics, each function may have different metrics group. So what about keeping it as it's now and reconsider whether we can share them when there are actually multiple user-defined functions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org