hequn8128 commented on a change in pull request #13097: URL: https://github.com/apache/flink/pull/13097#discussion_r467710965
########## File path: flink-python/src/main/java/org/apache/flink/datastream/runtime/functions/python/PickledKeySelector.java ########## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.datastream.runtime.functions.python; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.types.Row; + +import net.razorvine.pickle.Unpickler; + +/** + * PickledKeySelector is responsible for extracting the first filed of the input row as key. + * The input row is generated by python DataStream map function in the format of (key_selector.get_key(value), value) + * tuple2. + */ +public class PickledKeySelector implements KeySelector<Row, Object> { + + private Unpickler unpickler = null; + + @Override + public Object getKey(Row value) throws Exception { + + if (this.unpickler == null) { + unpickler = new Unpickler(); + } + + Object key = value.getField(0); + if (key instanceof byte[]) { Review comment: The key can be other types if key_by with type information. For example, `ds.key_by(MyKeySelector(), Types.INT())` ########## File path: flink-python/pyflink/datastream/data_stream.py ########## @@ -290,3 +339,39 @@ def _get_java_python_function_operator(self, func: Union[Function, FunctionWrapp output_type_info.get_java_type_info(), j_python_data_stream_function_info) return j_python_data_stream_scalar_function_operator, output_type_info + + +class KeyedStream(DataStream): Review comment: I find there are bugs in the methods of python `DataStream`, i.e., we should check DataStream types when perform `set_parallelism`, `name`, `uid`, etc. Please create another jira to fix the problem. ########## File path: flink-python/pyflink/datastream/tests/test_data_stream.py ########## @@ -140,15 +135,68 @@ def flat_map(value): flat_mapped_stream = ds.flat_map(flat_map, type_info=Types.ROW([Types.STRING(), Types.INT()])) - collect_util = DataStreamCollectUtil() - collect_util.collect(flat_mapped_stream) + self.collect_util.collect(flat_mapped_stream) self.env.execute('flat_map_test') - results = collect_util.results() + results = self.collect_util.results() expected = ['a,0', 'bdc,2', 'deeefg,4'] results.sort() expected.sort() self.assertEqual(expected, results) + def test_key_by(self): + element_collection = [('a', 0), ('b', 0), ('c', 1), ('d', 1), ('e', 2)] + self.env.set_parallelism(1) + ds = self.env.from_collection(element_collection, + type_info=Types.ROW([Types.STRING(), Types.INT()])) + + class AssertKeyMapFunction(MapFunction): + def __init__(self): + self.pre = None + + def map(self, value): + if value[0] == 'b': + assert self.pre == 'a' + if value[0] == 'd': + assert self.pre == 'c' + self.pre = value[0] + return value + + mapped_stream = ds.key_by(MyKeySelector()).map(AssertKeyMapFunction()) + self.collect_util.collect(mapped_stream) + self.env.execute('key_by_test') + results = self.collect_util.results() + expected = ["<Row('a', 0)>", "<Row('b', 0)>", "<Row('c', 1)>", "<Row('d', 1)>", + "<Row('e', 2)>"] + results.sort() + expected.sort() + self.assertEqual(expected, results) + + def test_key_by_map(self): Review comment: The two tests are duplicated? ########## File path: flink-python/pyflink/datastream/data_stream.py ########## @@ -233,6 +234,54 @@ def flat_map(self, func: Union[Callable, FlatMapFunction], type_info: TypeInform j_python_data_stream_scalar_function_operator )) + def key_by(self, key_selector: Union[Callable, KeySelector], + key_type_info: TypeInformation = None) -> 'KeyedStream': + """ + Creates a new KeyedStream that uses the provided key for partitioning its operator states. + + :param key_selector: The KeySelector to be used for extracting the key for partitioning. + :param key_type_info: The type information describing the key type. + :return: The DataStream with partitioned state(i.e. KeyedStream). + """ + if callable(key_selector): + key_selector = KeySelectorFunctionWrapper(key_selector) + if not isinstance(key_selector, (KeySelector, KeySelectorFunctionWrapper)): + raise TypeError("Parameter key_selector should be a type of KeySelector.") + + gateway = get_gateway() + PickledKeySelector = gateway.jvm \ + .org.apache.flink.datastream.runtime.functions.python.PickledKeySelector + j_output_type_info = self._j_data_stream.getTransformation().getOutputType() + output_type_info = typeinfo._from_java_type(j_output_type_info) + if key_type_info is None: + key_type_info = Types.PICKLED_BYTE_ARRAY() + generated_key_stream = KeyedStream(self.map(lambda x: (key_selector.get_key(x), x), + type_info=Types.ROW([key_type_info, + output_type_info])) + ._j_data_stream + .keyBy(PickledKeySelector())) + generated_key_stream._original_data_type_info = output_type_info + return generated_key_stream + + def _align_output_type(self) -> 'DataStream': Review comment: The method has never been used. ########## File path: flink-python/pyflink/datastream/data_stream.py ########## @@ -233,6 +234,54 @@ def flat_map(self, func: Union[Callable, FlatMapFunction], type_info: TypeInform j_python_data_stream_scalar_function_operator )) + def key_by(self, key_selector: Union[Callable, KeySelector], + key_type_info: TypeInformation = None) -> 'KeyedStream': + """ + Creates a new KeyedStream that uses the provided key for partitioning its operator states. + + :param key_selector: The KeySelector to be used for extracting the key for partitioning. + :param key_type_info: The type information describing the key type. + :return: The DataStream with partitioned state(i.e. KeyedStream). + """ + if callable(key_selector): + key_selector = KeySelectorFunctionWrapper(key_selector) + if not isinstance(key_selector, (KeySelector, KeySelectorFunctionWrapper)): + raise TypeError("Parameter key_selector should be a type of KeySelector.") + + gateway = get_gateway() + PickledKeySelector = gateway.jvm \ + .org.apache.flink.datastream.runtime.functions.python.PickledKeySelector + j_output_type_info = self._j_data_stream.getTransformation().getOutputType() + output_type_info = typeinfo._from_java_type(j_output_type_info) + if key_type_info is None: + key_type_info = Types.PICKLED_BYTE_ARRAY() + generated_key_stream = KeyedStream(self.map(lambda x: (key_selector.get_key(x), x), + type_info=Types.ROW([key_type_info, + output_type_info])) + ._j_data_stream + .keyBy(PickledKeySelector())) Review comment: .keyBy(PickledKeySelector(), key_type_info) ########## File path: flink-python/src/main/java/org/apache/flink/datastream/runtime/functions/python/PickledKeySelector.java ########## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.datastream.runtime.functions.python; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.types.Row; + +import net.razorvine.pickle.Unpickler; + +/** + * PickledKeySelector is responsible for extracting the first filed of the input row as key. Review comment: Use java annotation to describe Java class(e.g., PickledKeySelector, DataStream). ---------------------------------------------------------------- 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]
