[ https://issues.apache.org/jira/browse/FLINK-5886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019494#comment-16019494 ]
ASF GitHub Bot commented on FLINK-5886: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3838#discussion_r117724247 --- Diff: flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/datastream/PythonDataStream.java --- @@ -0,0 +1,262 @@ +/* + * 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.streaming.python.api.datastream; + +import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.core.fs.FileSystem.WriteMode; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.python.api.functions.PyKey; +import org.apache.flink.streaming.python.api.functions.PythonFilterFunction; +import org.apache.flink.streaming.python.api.functions.PythonFlatMapFunction; +import org.apache.flink.streaming.python.api.functions.PythonKeySelector; +import org.apache.flink.streaming.python.api.functions.PythonMapFunction; +import org.apache.flink.streaming.python.api.functions.PythonOutputSelector; +import org.apache.flink.streaming.python.api.functions.PythonSinkFunction; +import org.apache.flink.streaming.python.util.serialization.PythonSerializationSchema; +import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.python.core.PyObject; + +import java.io.IOException; +import java.util.ArrayList; + + +/** + * A {@code PythonDataStream} is a thin wrapper layer over {@link DataStream}, which represents a + * stream of elements of the same type. A {@code PythonDataStream} can be transformed into + * another {@code PythonDataStream} by applying various transformation functions, such as + * <ul> + * <li>{@link PythonDataStream#map} + * <li>{@link PythonDataStream#split} + * </ul> + * + * <p>A thin wrapper layer means that the functionality itself is performed by the + * {@link DataStream}, however instead of working directly with the streaming data sets, + * this layer handles Python wrappers (e.g. {@code PythonDataStream}) to comply with the + * Python standard coding styles.</p> + */ +@Public +public class PythonDataStream<D extends DataStream<PyObject>> { + protected final D stream; + + public PythonDataStream(D stream) { + this.stream = stream; + } + + /** + * A thin wrapper layer over {@link DataStream#union(DataStream[])}. + * + * @param streams + * The Python DataStreams to union output with. + * @return The {@link PythonDataStream}. + */ + @SafeVarargs + @SuppressWarnings("unchecked") + public final PythonDataStream union(PythonDataStream... streams) { + ArrayList<DataStream<PyObject>> dsList = new ArrayList<>(); + for (PythonDataStream ps : streams) { + dsList.add(ps.stream); + } + DataStream<PyObject>[] dsArray = new DataStream[dsList.size()]; + return new PythonDataStream(stream.union(dsList.toArray(dsArray))); + } + + /** + * A thin wrapper layer over {@link DataStream#split(OutputSelector)}. + * + * @param output_selector + * The user defined {@link OutputSelector} for directing the tuples. + * @return The {@link PythonSplitStream} + */ + public PythonSplitStream split(OutputSelector<PyObject> output_selector) throws IOException { + return new PythonSplitStream(this.stream.split(new PythonOutputSelector(output_selector))); + } + + /** + * A thin wrapper layer over {@link DataStream#filter(FilterFunction)}. + * + * @param filter + * The FilterFunction that is called for each element of the DataStream. + * @return The filtered {@link PythonDataStream}. + */ + public PythonSingleOutputStreamOperator filter(FilterFunction<PyObject> filter) throws IOException { + return new PythonSingleOutputStreamOperator(stream.filter(new PythonFilterFunction(filter))); + } + + /** + * A thin wrapper layer over {@link DataStream#map(MapFunction)}. + * + * @param mapper + * The MapFunction that is called for each element of the + * DataStream. + * @return The transformed {@link PythonDataStream}. + */ + public PythonDataStream<SingleOutputStreamOperator<PyObject>> map( + MapFunction<PyObject, PyObject> mapper) throws IOException { + return new PythonDataStream<>(stream.map(new PythonMapFunction(mapper))); + } + + /** + * A thin wrapper layer over {@link DataStream#flatMap(FlatMapFunction)}. + * + * @param flat_mapper + * The FlatMapFunction that is called for each element of the + * DataStream + * + * @return The transformed {@link PythonDataStream}. + */ + public PythonDataStream<SingleOutputStreamOperator<PyObject>> flat_map( + FlatMapFunction<PyObject, PyObject> flat_mapper) throws IOException { + return new PythonDataStream<>(stream.flatMap(new PythonFlatMapFunction(flat_mapper))); + } + + /** + * A thin wrapper layer over {@link DataStream#keyBy(KeySelector)}. + * + * @param selector + * The KeySelector to be used for extracting the key for partitioning + * @return The {@link PythonDataStream} with partitioned state (i.e. {@link PythonKeyedStream}) + */ + public PythonKeyedStream key_by(KeySelector<PyObject, PyKey> selector) throws IOException { + return new PythonKeyedStream(stream.keyBy(new PythonKeySelector(selector))); + } + + /** + * A thin wrapper layer over {@link DataStream#print()}. + */ + @PublicEvolving + public void print() { stream.print(); } + + /** + * A thin wrapper layer over {@link DataStream#writeAsText(java.lang.String)}. + * + * @param path + * The path pointing to the location the text file is written to. + */ + @PublicEvolving + public void write_as_text(String path) { stream.writeAsText(path); } --- End diff -- `stream.print();` should be on a separate line. > Python API for streaming applications > ------------------------------------- > > Key: FLINK-5886 > URL: https://issues.apache.org/jira/browse/FLINK-5886 > Project: Flink > Issue Type: New Feature > Components: Python API > Reporter: Zohar Mizrahi > Assignee: Zohar Mizrahi > > A work in progress to provide python interface for Flink streaming APIs. The > core technology is based on jython and thus imposes two limitations: a. user > defined functions cannot use python extensions. b. the python version is 2.x > The branch is based on Flink release 1.2.0, as can be found here: > https://github.com/zohar-pm/flink/tree/python-streaming > In order to test it, someone can use IntelliJ IDE. Assuming IntelliJ was > setup properly (see: > https://ci.apache.org/projects/flink/flink-docs-release-1.3/internals/ide_setup.html), > one can run/debug {{org.apache.flink.python.api.PythonStreamBinderTest}}, > which in return will execute all the tests under > {{/Users/zohar/dev/pm-flink/flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/streaming}} -- This message was sent by Atlassian JIRA (v6.3.15#6346)