dianfu commented on a change in pull request #12902:
URL: https://github.com/apache/flink/pull/12902#discussion_r508938695



##########
File path: flink-python/pyflink/table/table_result.py
##########
@@ -151,6 +156,45 @@ def get_result_kind(self):
         """
         return 
ResultKind._from_j_result_kind(self._j_table_result.getResultKind())
 
+    def collect(self):
+        """
+        Get the result contents as a closeable row iterator.
+
+        Note:
+
+        For SELECT operation, the job will not be finished unless all result 
data has been
+        collected. So we should actively close the job to avoid resource leak 
through
+        CloseableIterator#close method. Calling CloseableIterator#close method 
will cancel the job
+        and release related resources.
+
+        For DML operation, Flink dose not support getting the real affected 
row count now. So the

Review comment:
       ```suggestion
           For DML operation, Flink does not support getting the real affected 
row count now. So the
   ```

##########
File path: flink-python/pyflink/table/tests/test_table_environment_api.py
##########
@@ -454,6 +458,145 @@ def test_to_retract_stream(self):
                     "(True, <Row(2, 'Hello')>)"]
         self.assertEqual(result, expected)
 
+    def test_collect_for_all_data_types(self):
+
+        def collect_from_source(t_env, expected_result):
+            source = t_env.from_elements([(1, None, 1, True, 32767, 
-2147483648, 1.23, 1.98932,
+                                           bytearray(b'pyflink'), 'pyflink',
+                                           datetime.date(2014, 9, 13),
+                                           datetime.time(hour=12, minute=0, 
second=0,
+                                                         microsecond=123000),
+                                           datetime.datetime(2018, 3, 11, 3, 
0, 0, 123000),
+                                           [Row(['pyflink']), 
Row(['pyflink']), Row(['pyflink'])],
+                                           {1: Row(['flink']), 2: 
Row(['pyflink'])},
+                                           
decimal.Decimal('1000000000000000000.05'),
+                                           
decimal.Decimal('1000000000000000000.0599999999999999989'
+                                                           '9999999999'))],
+                                         DataTypes.ROW([DataTypes.FIELD("a", 
DataTypes.BIGINT()),
+                                                        DataTypes.FIELD("b", 
DataTypes.BIGINT()),
+                                                        DataTypes.FIELD("c", 
DataTypes.TINYINT()),
+                                                        DataTypes.FIELD("d", 
DataTypes.BOOLEAN()),
+                                                        DataTypes.FIELD("e", 
DataTypes.SMALLINT()),
+                                                        DataTypes.FIELD("f", 
DataTypes.INT()),
+                                                        DataTypes.FIELD("g", 
DataTypes.FLOAT()),
+                                                        DataTypes.FIELD("h", 
DataTypes.DOUBLE()),
+                                                        DataTypes.FIELD("i", 
DataTypes.BYTES()),
+                                                        DataTypes.FIELD("j", 
DataTypes.STRING()),
+                                                        DataTypes.FIELD("k", 
DataTypes.DATE()),
+                                                        DataTypes.FIELD("l", 
DataTypes.TIME()),
+                                                        DataTypes.FIELD("m",
+                                                                        
DataTypes.TIMESTAMP(3)),
+                                                        DataTypes.FIELD("n", 
DataTypes.ARRAY(
+                                                            
DataTypes.ROW([DataTypes.FIELD('ss2',
+                                                                           
DataTypes.STRING())]))),
+                                                        DataTypes.FIELD("o", 
DataTypes.MAP(
+                                                            
DataTypes.BIGINT(), DataTypes.ROW(
+                                                                
[DataTypes.FIELD('ss',
+                                                                 
DataTypes.STRING())]))),
+                                                        DataTypes.FIELD("p",
+                                                                        
DataTypes.DECIMAL(38, 18)),
+                                                        DataTypes.FIELD("q",
+                                                        DataTypes.DECIMAL(38, 
18))]))
+            table_result = source.execute()
+            with table_result.collect() as result:
+                collected_result = []
+                for i in result:
+                    collected_result.append(i)
+                self.assertEqual(expected_result, collected_result)
+
+        legacy_planer_t_env = self.t_env
+        expected_result_legacy_planner = [Row([1, None, 1, True, 32767, 
-2147483648, 1.23,
+                                          1.98932, bytearray(b'pyflink'), 
'pyflink',
+                                          datetime.date(2014, 9, 13), 
datetime.time(12, 0),
+                                          datetime.datetime(2018, 3, 11, 3, 0, 
0, 123000),
+                                          [Row(['[pyflink]']), 
Row(['[pyflink]']),
+                                           Row(['[pyflink]'])],
+                                          {1: Row(['[flink]']), 2: 
Row(['[pyflink]'])},
+                                          
decimal.Decimal('1000000000000000000.05'),
+                                          
decimal.Decimal('1000000000000000000.05999999999999999899'
+                                                          '999999999')])]
+        collect_from_source(legacy_planer_t_env, 
expected_result_legacy_planner)
+
+        blink_planer_t_env = StreamTableEnvironment.create(
+            StreamExecutionEnvironment.get_execution_environment(),
+            environment_settings=EnvironmentSettings.new_instance()
+            .in_streaming_mode().use_blink_planner().build())
+        blink_planer_t_env.get_config().get_configuration().set_string(
+            "taskmanager.memory.task.off-heap.size", "80mb")
+        expected_result_blink_planner = [Row([1, None, 1, True, 32767, 
-2147483648, 1.23,
+                                         1.98932, bytearray(b'pyflink'), 
'pyflink',
+                                         datetime.date(2014, 9, 13),
+                                              datetime.time(12, 0, 0, 123000),
+                                         datetime.datetime(2018, 3, 11, 3, 0, 
0, 123000),
+                                         [Row(['[pyflink]']), 
Row(['[pyflink]']),
+                                          Row(['[pyflink]'])],
+                                         {1: Row(['[flink]']), 2: 
Row(['[pyflink]'])},
+                                         
decimal.Decimal('1000000000000000000.050000000000000000'),
+                                         
decimal.Decimal('1000000000000000000.059999999999999999')])
+                                         ]
+        collect_from_source(blink_planer_t_env, expected_result_blink_planner)
+
+    def test_collect_with_retract(self):
+
+        def collect_retract_result_with(t_env, expected_row_kinds):
+            element_data = [(1, 2, 'a'),
+                            (3, 4, 'b'),
+                            (5, 6, 'a'),
+                            (7, 8, 'b')]
+            field_names = ['a', 'b', 'c']
+            source = t_env.from_elements(element_data, field_names)
+            table_result = t_env.execute_sql(
+                "SELECT SUM(a), c FROM %s group by c" % source)
+            with table_result.collect() as result:
+                collected_result = []
+                for i in result:
+                    collected_result.append(i)
+
+                collected_result = [str(result) + ',' + 
str(result.get_row_kind())
+                                    for result in collected_result]
+                expected_result = [Row([1, 'a']), Row([1, 'a']), Row([6, 
'a']), Row([3, 'b']),
+                                   Row([3, 'b']), Row([10, 'b'])]
+                for i in range(len(expected_result)):
+                    expected_result[i] = str(expected_result[i]) + ',' + 
str(expected_row_kinds[i])
+                expected_result.sort()
+                collected_result.sort()
+                self.assertEqual(expected_result, collected_result)
+
+        expected_row_kinds_legacy = [RowKind.INSERT, RowKind.DELETE, 
RowKind.INSERT, RowKind.INSERT,
+                                     RowKind.DELETE, RowKind.INSERT]
+
+        # Test legacy planner
+        collect_retract_result_with(self.t_env, expected_row_kinds_legacy)
+
+        # Test blink planner

Review comment:
       split the tests for legacy planner and blink planner into two test cases?

##########
File path: flink-python/pyflink/table/table_result.py
##########
@@ -151,6 +156,45 @@ def get_result_kind(self):
         """
         return 
ResultKind._from_j_result_kind(self._j_table_result.getResultKind())
 
+    def collect(self):

Review comment:
       ```suggestion
       def collect(self) -> "CloseableIterator":
   ```

##########
File path: flink-python/pyflink/table/table_result.py
##########
@@ -173,3 +217,49 @@ def print(self):
         .. versionadded:: 1.11.0
         """
         self._j_table_result.print()
+
+
+class CloseableIterator(object):
+    """
+    Representing an Iterator that is also auto closeable.
+    """
+    def __init__(self, j_closeable_iterator, field_data_types):
+        self._j_closeable_iterator = j_closeable_iterator
+        self._j_field_data_types = field_data_types
+        self._data_types = [_from_java_type(j_field_data_type)
+                            for j_field_data_type in self._j_field_data_types]
+
+    def __iter__(self):
+        return self
+
+    def __next__(self):
+        if not self._j_closeable_iterator.hasNext():
+            raise StopIteration("No more data.")
+        gateway = get_gateway()
+        pickle_bytes = gateway.jvm.PythonBridgeUtils. \
+            getPickledBytesFromRow(self._j_closeable_iterator.next(),
+                                   self._j_field_data_types)
+        row_kind = RowKind(int.from_bytes(pickle_bytes[0], byteorder='big', 
signed=False))
+        pickle_bytes = list(pickle_bytes[1:])
+        field_data = zip(pickle_bytes, self._data_types)
+        fields = []
+        for data, field_type in field_data:
+            if len(data) == 0:
+                fields.append(None)
+            else:
+                fields.append(java_to_python_converter(data, field_type))
+        result_row = Row(fields)
+        result_row.set_row_kind(row_kind)
+        return result_row
+
+    def next(self):
+        return self.__next__()
+
+    def close(self):
+        self._j_closeable_iterator.close()
+
+    def __enter__(self):
+        return self

Review comment:
       ```suggestion
           pass
   ```




----------------------------------------------------------------
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:
[email protected]


Reply via email to