Skip to content

Commit

Permalink
[FLINK-12311][python] Add base python framework and Add Scan, Project…
Browse files Browse the repository at this point in the history
…ion, and Filter operator support

This closes apache#8267

make travis tests green

fix documents, refine shell file name and optimize program logic.

Adjust the code examples in python documents to be consistent with scala documents.

Refactor flink-python project structure, remove java file from flink-python package.

delete unnecessary dependency and plugins.
  • Loading branch information
WeiZhong94 authored and sunjincheng121 committed May 6, 2019
1 parent 4e505c6 commit a09bd4c
Show file tree
Hide file tree
Showing 33 changed files with 1,651 additions and 1 deletion.
5 changes: 5 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,9 @@ tmp
*.iml
*.swp
*.jar
*.zip
*.log
*.pyc
.DS_Store
build-target
flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/avro/
Expand All @@ -23,6 +25,9 @@ flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/gener
flink-runtime-web/web-dashboard/node/
flink-runtime-web/web-dashboard/node_modules/
flink-runtime-web/web-dashboard/web/
flink-python/dist/
flink-python/build/
flink-python/pyflink.egg-info/
atlassian-ide-plugin.xml
out/
/docs/api
Expand Down
75 changes: 75 additions & 0 deletions docs/dev/table/tableApi.md
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,81 @@ val result = orders.where('b === "red")
</tr>
</tbody>
</table>
</div>
<div data-lang="python" markdown="1">

<table class="table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 20%">Operators</th>
<th class="text-center">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>
<strong>Scan</strong><br>
<span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
</td>
<td>
<p>Similar to the FROM clause in a SQL query. Performs a scan of a registered table.</p>
{% highlight python %}
orders = table_env.scan("Orders");
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Select</strong><br>
<span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
</td>
<td>
<p>Similar to a SQL SELECT statement. Performs a select operation.</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.select("a, c as d");
{% endhighlight %}
<p>You can use star (<code>*</code>) to act as a wild card, selecting all of the columns in the table.</p>
{% highlight python %}
result = orders.select("*");
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Alias</strong><br>
<span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
</td>
<td>
<p>Renames fields.</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.alias("x, y, z, t");
{% endhighlight %}
</td>
</tr>

<tr>
<td>
<strong>Where / Filter</strong><br>
<span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span>
</td>
<td>
<p>Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.where("b === 'red'");
{% endhighlight %}
or
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.filter("a % 2 === 0");
{% endhighlight %}
</td>
</tr>
</tbody>
</table>

</div>
</div>

Expand Down
75 changes: 75 additions & 0 deletions docs/dev/table/tableApi.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,81 @@ val result = orders.where('b === "red")
</tr>
</tbody>
</table>
</div>
<div data-lang="python" markdown="1">

<table class="table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 20%">操作</th>
<th class="text-center">描述</th>
</tr>
</thead>
<tbody>
<tr>
<td>
<strong>Scan</strong><br>
<span class="label label-primary">批处理</span> <span class="label label-primary">流处理</span>
</td>
<td>
<p>类似于SQL请求中的FROM子句,将一个环境中已注册的表转换成Table对象。</p>
{% highlight python %}
orders = table_env.scan("Orders");
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Select</strong><br>
<span class="label label-primary">批处理</span> <span class="label label-primary">流处理</span>
</td>
<td>
<p>类似于SQL请求中的SELECT子句,执行一个select操作。</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.select("a, c as d");
{% endhighlight %}
<p>您可以使用星号 (<code>*</code>) 表示选择表中的所有列。</p>
{% highlight python %}
result = orders.select("*");
{% endhighlight %}
</td>
</tr>
<tr>
<td>
<strong>Alias</strong><br>
<span class="label label-primary">批处理</span> <span class="label label-primary">流处理</span>
</td>
<td>
<p>重命名字段。</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.alias("x, y, z, t");
{% endhighlight %}
</td>
</tr>

<tr>
<td>
<strong>Where / Filter</strong><br>
<span class="label label-primary">批处理</span> <span class="label label-primary">流处理</span>
</td>
<td>
<p>类似于SQL请求中的WHERE子句,过滤掉表中不满足条件的行。</p>
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.where("b === 'red'");
{% endhighlight %}
or
{% highlight python %}
orders = table_env.scan("Orders");
result = orders.filter("a % 2 === 0");
{% endhighlight %}
</td>
</tr>
</tbody>
</table>

</div>
</div>

Expand Down
7 changes: 7 additions & 0 deletions flink-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,13 @@ under the License.
<artifactId>flink-shaded-guava</artifactId>
</dependency>

<!-- Python API dependencies -->
<dependency>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>
<version>0.10.8.1</version>
</dependency>

<!-- ================== test dependencies ================== -->

<dependency>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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:https://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.api.python;

import py4j.GatewayServer;

import java.io.DataOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.net.InetAddress;

/**
* The Py4j Gateway Server provides RPC service for user's python process.
*/
public class PythonGatewayServer {

/**
* <p>
* Main method to start a local GatewayServer on a ephemeral port.
* It tells python side via a file.
*
* See: py4j.GatewayServer.main()
* </p>
*/
public static void main(String[] args) throws IOException {
InetAddress localhost = InetAddress.getLoopbackAddress();
GatewayServer gatewayServer = new GatewayServer.GatewayServerBuilder()
.javaPort(0)
.javaAddress(localhost)
.build();
gatewayServer.start();

int boundPort = gatewayServer.getListeningPort();
if (boundPort == -1) {
System.out.println("GatewayServer failed to bind; exiting");
System.exit(1);
}

// Tells python side the port of our java rpc server
String handshakeFilePath = System.getenv("_PYFLINK_CONN_INFO_PATH");
File handshakeFile = new File(handshakeFilePath);
if (handshakeFile.createNewFile()) {
FileOutputStream fileOutputStream = new FileOutputStream(handshakeFile);
DataOutputStream stream = new DataOutputStream(fileOutputStream);
stream.writeInt(boundPort);
stream.close();
fileOutputStream.close();
} else {
System.out.println("Can't create handshake file: " + handshakeFilePath + ", now exit...");
return;
}

// Exit on EOF or broken pipe. This ensures that the server dies
// if its parent program dies.
while (System.in.read() != -1) {
// Do nothing
}
gatewayServer.shutdown();
}
}
6 changes: 6 additions & 0 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -568,6 +568,12 @@ under the License.
<projectName>Apache Flink</projectName>
</transformer>
</transformers>
<relocations>
<relocation>
<pattern>py4j</pattern>
<shadedPattern>org.apache.flink.api.python.py4j</shadedPattern>
</relocation>
</relocations>
</configuration>
</execution>
<execution>
Expand Down
7 changes: 7 additions & 0 deletions flink-dist/src/main/assemblies/opt.xml
Original file line number Diff line number Diff line change
Expand Up @@ -178,4 +178,11 @@
<fileMode>0644</fileMode>
</file>
</files>
<fileSets>
<fileSet>
<directory>../flink-python/lib/</directory>
<outputDirectory>opt/python/</outputDirectory>
<fileMode>0644</fileMode>
</fileSet>
</fileSets>
</assembly>
54 changes: 54 additions & 0 deletions flink-dist/src/main/flink-bin/bin/pyflink-gateway-server.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
#!/usr/bin/env bash
################################################################################
# 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:https://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.
################################################################################

# =====================================================================
bin=`dirname "$0"`
bin=`cd "$bin"; pwd`

. "$bin"/config.sh

if [ "$FLINK_IDENT_STRING" = "" ]; then
FLINK_IDENT_STRING="$USER"
fi

FLINK_CLASSPATH=`constructFlinkClassPath`

ARGS=()

while [[ $# -gt 0 ]]
do
key="$1"
case $key in
-c|--class)
DRIVER=$2
shift
shift
;;
*)
ARGS+=("$1")
shift
;;
esac
done

log=$FLINK_LOG_DIR/flink-$FLINK_IDENT_STRING-client-$HOSTNAME.log
log_setting=(-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4j-cli.properties -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/logback.xml)

TABLE_JAR_PATH=`echo "$FLINK_ROOT_DIR"/opt/flink-table*.jar`
exec $JAVA_RUN $JVM_ARGS "${log_setting[@]}" -cp ${FLINK_CLASSPATH}:${TABLE_JAR_PATH} ${DRIVER} ${ARGS[@]}
1 change: 1 addition & 0 deletions flink-dist/src/main/resources/META-INF/NOTICE
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ See bundled license files for details.
- com.esotericsoftware.kryo:kryo:2.24.0
- com.esotericsoftware.minlog:minlog:1.2
- org.clapper:grizzled-slf4j_2.11:1.3.2
- net.sf.py4j:py4j:0.10.8.1

The following dependencies all share the same BSD license which you find under licenses/LICENSE.scala.

Expand Down
26 changes: 26 additions & 0 deletions flink-dist/src/main/resources/META-INF/licenses/LICENSE.py4j
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
Copyright (c) 2009-2018, Barthelemy Dagenais and individual contributors. All
rights reserved.

Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are met:

- Redistributions of source code must retain the above copyright notice, this
list of conditions and the following disclaimer.

- Redistributions in binary form must reproduce the above copyright notice,
this list of conditions and the following disclaimer in the documentation
and/or other materials provided with the distribution.

- The name of the author may not be used to endorse or promote products
derived from this software without specific prior written permission.

THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
Loading

0 comments on commit a09bd4c

Please sign in to comment.