def test_partition_custom(self): ds = self.env.from_collection([('a', 0), ('b', 0), ('c', 1), ('d', 1), ('e', 2), ('f', 7), ('g', 7), ('h', 8), ('i', 8), ('j', 9)], type_info=Types.ROW( [Types.STRING(), Types.INT()])) expected_num_partitions = 5 def my_partitioner(key, num_partitions): assert expected_num_partitions, num_partitions return key % num_partitions partitioned_stream = ds.map(lambda x: x, output_type=Types.ROW([Types.STRING(), Types.INT()]))\ .set_parallelism(4).partition_custom(my_partitioner, lambda x: x[1]) JPartitionCustomTestMapFunction = get_gateway().jvm\ .org.apache.flink.python.util.PartitionCustomTestMapFunction test_map_stream = DataStream( partitioned_stream._j_data_stream.map( JPartitionCustomTestMapFunction())) test_map_stream.set_parallelism(expected_num_partitions).add_sink( self.test_sink) self.env.execute('test_partition_custom')
def from_source(self, source: Source, watermark_strategy: WatermarkStrategy, source_name: str, type_info: TypeInformation = None) -> 'DataStream': """ Adds a data :class:`~pyflink.datastream.connectors.Source` to the environment to get a :class:`~pyflink.datastream.DataStream`. The result will be either a bounded data stream (that can be processed in a batch way) or an unbounded data stream (that must be processed in a streaming way), based on the boundedness property of the source. This method takes an explicit type information for the produced data stream, so that callers can define directly what type/serializer will be used for the produced stream. For sources that describe their produced type, the parameter type_info should not be specified to avoid specifying the produced type redundantly. .. versionadded:: 1.13.0 """ if type_info: j_type_info = type_info.get_java_type_info() else: j_type_info = None j_data_stream = self._j_stream_execution_environment.fromSource( source.get_java_function(), watermark_strategy._j_watermark_strategy, source_name, j_type_info) return DataStream(j_data_stream=j_data_stream)
def create_input(self, input_format: InputFormat, type_info: Optional[TypeInformation] = None): """ Create an input data stream with InputFormat. If the input_format needs a well-defined type information (e.g. Avro's generic record), you can either explicitly use type_info argument or use InputFormats implementing ResultTypeQueryable. :param input_format: The input format to read from. :param type_info: Optional type information to explicitly declare output type. .. versionadded:: 1.16.0 """ input_type_info = type_info if input_type_info is None and isinstance(input_format, ResultTypeQueryable): input_type_info = cast(ResultTypeQueryable, input_format).get_produced_type() if input_type_info is None: j_data_stream = self._j_stream_execution_environment.createInput( input_format.get_java_object()) else: j_data_stream = self._j_stream_execution_environment.createInput( input_format.get_java_object(), input_type_info.get_java_type_info()) return DataStream(j_data_stream=j_data_stream)
def collect(self, data_stream: DataStream): gateway = get_gateway() self._is_python_objects = data_stream.get_type( ) == Types.PICKLED_BYTE_ARRAY() self._j_data_stream_test_collect_sink = gateway.jvm \ .org.apache.flink.python.util.DataStreamTestCollectSink(self._is_python_objects) data_stream._j_data_stream.addSink( self._j_data_stream_test_collect_sink)
def apply(self, ds): jvm = get_gateway().jvm if _check_if_row_data_type(ds): return ds j_map_function = jvm.org.apache.flink.python.util.PythonConnectorUtils \ .RowRowMapper(_to_java_data_type(row_type)) return DataStream(ds._j_data_stream.process(j_map_function))
def _from_collection(self, elements: List[Any], type_info: TypeInformation = None) -> DataStream: temp_file = tempfile.NamedTemporaryFile(delete=False, dir=tempfile.mkdtemp()) serializer = self.serializer try: with temp_file: # dumps elements to a temporary file by pickle serializer. serializer.serialize(elements, temp_file) gateway = get_gateway() # if user does not defined the element data types, read the pickled data as a byte array # list. if type_info is None: j_objs = gateway.jvm.PythonBridgeUtils.readPickledBytes( temp_file.name) out_put_type_info = Types.PICKLED_BYTE_ARRAY( ) # type: TypeInformation else: j_objs = gateway.jvm.PythonBridgeUtils.readPythonObjects( temp_file.name) out_put_type_info = type_info # Since flink python module depends on table module, we can make use of utils of it when # implementing python DataStream API. PythonTableUtils = gateway.jvm\ .org.apache.flink.table.planner.utils.python.PythonTableUtils execution_config = self._j_stream_execution_environment.getConfig() j_input_format = PythonTableUtils.getCollectionInputFormat( j_objs, out_put_type_info.get_java_type_info(), execution_config) JInputFormatSourceFunction = gateway.jvm.org.apache.flink.streaming.api.functions.\ source.InputFormatSourceFunction JBoundedness = gateway.jvm.org.apache.flink.api.connector.source.Boundedness j_data_stream_source = invoke_method( self._j_stream_execution_environment, "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment", "addSource", [ JInputFormatSourceFunction( j_input_format, out_put_type_info.get_java_type_info()), "Collection Source", out_put_type_info.get_java_type_info(), JBoundedness.BOUNDED ], [ "org.apache.flink.streaming.api.functions.source.SourceFunction", "java.lang.String", "org.apache.flink.api.common.typeinfo.TypeInformation", "org.apache.flink.api.connector.source.Boundedness" ]) j_data_stream_source.forceNonParallel() return DataStream(j_data_stream=j_data_stream_source) finally: os.unlink(temp_file.name)
def read_text_file(self, file_path: str, charset_name: str = "UTF-8") -> DataStream: """ Reads the given file line-by-line and creates a DataStream that contains a string with the contents of each such line. The charset with the given name will be used to read the files. Note that this interface is not fault tolerant that is supposed to be used for test purpose. :param file_path: The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") :param charset_name: The name of the character set used to read the file. :return: The DataStream that represents the data read from the given file as text lines. """ return DataStream(self._j_stream_execution_environment .readTextFile(file_path, charset_name))
def add_source(self, source_func: SourceFunction, source_name: str = 'Custom Source', type_info: TypeInformation = None) -> 'DataStream': """ Adds a data source to the streaming topology. :param source_func: the user defined function. :param source_name: name of the data source. Optional. :param type_info: type of the returned stream. Optional. :return: the data stream constructed. """ j_type_info = type_info.get_java_type_info() if type_info is not None else None j_data_stream = self._j_stream_execution_environment.addSource(source_func .get_java_function(), source_name, j_type_info) return DataStream(j_data_stream=j_data_stream)
def _from_collection(self, elements: List[Any], type_info: TypeInformation = None) -> DataStream: temp_file = tempfile.NamedTemporaryFile(delete=False, dir=tempfile.mkdtemp()) serializer = self.serializer try: with temp_file: # dumps elements to a temporary file by pickle serializer. serializer.dump_to_stream(elements, temp_file) gateway = get_gateway() # if user does not defined the element data types, read the pickled data as a byte array # list. if type_info is None: j_objs = gateway.jvm.PythonBridgeUtils.readPickledBytes( temp_file.name) out_put_type_info = PickledBytesTypeInfo.PICKLED_BYTE_ARRAY_TYPE_INFO( ) else: j_objs = gateway.jvm.PythonBridgeUtils.readPythonObjects( temp_file.name, False) out_put_type_info = type_info # Since flink python module depends on table module, we can make use of utils of it when # implementing python DataStream API. PythonTableUtils = gateway.jvm\ .org.apache.flink.table.planner.utils.python.PythonTableUtils execution_config = self._j_stream_execution_environment.getConfig() j_input_format = PythonTableUtils.getCollectionInputFormat( j_objs, out_put_type_info.get_java_type_info(), execution_config) j_data_stream_source = self._j_stream_execution_environment.createInput( j_input_format, out_put_type_info.get_java_type_info()) j_data_stream_source.forceNonParallel() return DataStream(j_data_stream=j_data_stream_source) finally: os.unlink(temp_file.name)