You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/06 05:07:14 UTC

[GitHub] [flink] dianfu commented on a change in pull request #11960: [FLINK-12717][python] Support running PyFlink on Windows

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



##########
File path: flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java
##########
@@ -159,7 +159,14 @@ public void open() throws Exception {
 	@Override
 	public void close() throws Exception {
 		try {
-			invokeFinishBundle();
+			try {

Review comment:
       What's the purpose of this change

##########
File path: flink-python/src/main/java/org/apache/flink/python/util/ResourceUtil.java
##########
@@ -29,10 +29,17 @@
  */
 public class ResourceUtil {
 
-	public static final String PYFLINK_UDF_RUNNER = "pyflink-udf-runner.sh";
+	public static final String PYFLINK_UDF_RUNNER_SH = "pyflink-udf-runner.sh";
+	public static final String PYFLINK_UDF_RUNNER_BAT = "pyflink-udf-runner.bat";
 
 	public static File extractUdfRunner(String tmpdir) throws IOException, InterruptedException {
-		File file = new File(tmpdir, PYFLINK_UDF_RUNNER);
+		File file;
+		// This program can not depend any other dependencies, so we check the operating system without any utils.

Review comment:
       Why this program cannot depend on other dependencies? 
   `OperatingSystem.isWindows()` cannot also be used?

##########
File path: flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java
##########
@@ -127,20 +130,36 @@ public void open() throws Exception {
 	}
 
 	@Override
-	public void close() {
-		FileUtils.deleteDirectoryQuietly(new File(baseDirectory));
-		if (shutdownHook != null) {
-			ShutdownHookUtil.removeShutdownHook(
-				shutdownHook, ProcessPythonEnvironmentManager.class.getSimpleName(), LOG);
-			shutdownHook = null;
+	public void close() throws Exception {
+		try {
+			int i = 0;
+			while (i < CHECK_TIMEOUT / CHECK_INTERVAL) {
+				try {
+					i++;
+					FileUtils.deleteDirectory(new File(baseDirectory));
+				} catch (Throwable t) {
+					if (i == CHECK_TIMEOUT / CHECK_INTERVAL) {
+						LOG.error("Clean the temporary directory of Python UDF worker failed.", t);
+						break;
+					}
+				}
+				Thread.sleep(CHECK_INTERVAL);
+			}
+		} finally {
+			if (shutdownHook != null) {
+				ShutdownHookUtil.removeShutdownHook(
+					shutdownHook, ProcessPythonEnvironmentManager.class.getSimpleName(), LOG);
+				shutdownHook = null;
+			}
+			LOG.info("Python environment manager is closing. Now print the content of boot log:\n" + getBootLog());

Review comment:
       why we need to print the log from boot log?

##########
File path: flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java
##########
@@ -127,20 +130,36 @@ public void open() throws Exception {
 	}
 
 	@Override
-	public void close() {
-		FileUtils.deleteDirectoryQuietly(new File(baseDirectory));
-		if (shutdownHook != null) {
-			ShutdownHookUtil.removeShutdownHook(
-				shutdownHook, ProcessPythonEnvironmentManager.class.getSimpleName(), LOG);
-			shutdownHook = null;
+	public void close() throws Exception {
+		try {
+			int i = 0;
+			while (i < CHECK_TIMEOUT / CHECK_INTERVAL) {
+				try {
+					i++;
+					FileUtils.deleteDirectory(new File(baseDirectory));
+				} catch (Throwable t) {
+					if (i == CHECK_TIMEOUT / CHECK_INTERVAL) {
+						LOG.error("Clean the temporary directory of Python UDF worker failed.", t);
+						break;
+					}
+				}
+				Thread.sleep(CHECK_INTERVAL);
+			}
+		} finally {
+			if (shutdownHook != null) {
+				ShutdownHookUtil.removeShutdownHook(
+					shutdownHook, ProcessPythonEnvironmentManager.class.getSimpleName(), LOG);
+				shutdownHook = null;
+			}
+			LOG.info("Python environment manager is closing. Now print the content of boot log:\n" + getBootLog());
 		}
 	}
 
 	@Override
 	public RunnerApi.Environment createEnvironment() throws IOException, InterruptedException {
 		Map<String, String> env = constructEnvironmentVariables();
-		ResourceUtil.extractUdfRunner(baseDirectory);
-		String pythonWorkerCommand = String.join(File.separator, baseDirectory, "pyflink-udf-runner.sh");
+		File runnerScript = ResourceUtil.extractUdfRunner(baseDirectory);
+		String pythonWorkerCommand = String.join(File.separator, baseDirectory, runnerScript.getName());

Review comment:
       ```suggestion
   		String pythonWorkerCommand = runnerScript.getPath();
   ```
   
   Besides, the variable `pythonWorkerCommand` could be removed.

##########
File path: flink-python/pyflink/pyflink_gateway_server.py
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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.
+################################################################################
+import argparse
+import getpass
+import glob
+import os
+import platform
+import re
+import signal
+import socket
+import sys
+from collections import namedtuple
+from string import Template
+from subprocess import Popen, PIPE, check_output
+
+from pyflink.find_flink_home import _find_flink_home, _find_flink_source_root
+
+
+def on_windows():
+    return platform.system() == "Windows"
+
+
+def find_java_executable():
+    java_executable = "java.exe" if on_windows() else "java"
+    flink_home = _find_flink_home()
+    flink_conf_path = os.path.join(flink_home, "conf", "flink-conf.yaml")
+    java_home = None
+
+    if os.path.isfile(flink_conf_path):
+        with open(flink_conf_path, "r") as f:
+            flink_conf_yaml = f.read()
+        java_homes = re.findall(r'^[ ]*env\.java\.home[ ]*: ([^#]*).*$', flink_conf_yaml)
+        if len(java_homes) > 1:
+            java_home = java_homes[len(java_homes) - 1].strip()
+
+    if java_home is None and "JAVA_HOME" in os.environ:
+        java_home = os.environ["JAVA_HOME"]
+
+    if java_home is not None:
+        java_executable = os.path.join(java_home, "bin", java_executable)
+
+    return java_executable
+
+
+def construct_log_settings():
+    templates = [
+        "-Dlog.file=${flink_log_dir}/flink-${flink_ident_string}-python-${hostname}.log",
+        "-Dlog4j.configuration=${flink_conf_dir}/log4j-cli.properties",
+        "-Dlog4j.configurationFile=${flink_conf_dir}/log4j-cli.properties",
+        "-Dlogback.configurationFile=${flink_conf_dir}/logback.xml"
+    ]
+
+    flink_home = _find_flink_home()
+    flink_conf_dir = os.path.join(flink_home, "conf")
+    flink_log_dir = os.path.join(flink_home, "log")
+    if "FLINK_IDENT_STRING" in os.environ:
+        flink_ident_string = os.environ["FLINK_IDENT_STRING"]
+    else:
+        flink_ident_string = getpass.getuser()
+    hostname = socket.gethostname()
+    log_settings = []
+    for template in templates:
+        log_settings.append(Template(template).substitute(
+            flink_conf_dir=flink_conf_dir,
+            flink_log_dir=flink_log_dir,
+            flink_ident_string=flink_ident_string,
+            hostname=hostname))
+    return log_settings
+
+
+def construct_classpath():
+    flink_home = _find_flink_home()
+    if on_windows():
+        # The command length is limited on Windows. To avoid the problem we should shorten the
+        # command length as much as possible.
+        lib_jars = os.path.join(flink_home, "lib", "*")
+    else:
+        lib_jars = os.pathsep.join(glob.glob(os.path.join(flink_home, "lib", "*.jar")))
+
+    flink_python_jars = glob.glob(os.path.join(flink_home, "opt", "flink-python*.jar"))
+    if len(flink_python_jars) < 1:
+        print("The flink-python jar is not found in the opt folder of the FLINK_HOME: %s" %
+              flink_home)
+        return lib_jars
+    flink_python_jar = flink_python_jars[0]
+
+    return os.pathsep.join([lib_jars, flink_python_jar])
+
+
+def download_apache_avro():
+    """
+    Currently we need to download the Apache Avro manually to avoid test failure caused by the avro
+    format sql jar. See https://issues.apache.org/jira/browse/FLINK-17417. If the issue is fixed,
+    this method could be removed. Using maven command copy the jars in repository to avoid accessing
+    external network.
+    """
+    flink_source_root = _find_flink_source_root()
+    avro_jar_pattern = os.path.join(
+        flink_source_root, "flink-formats", "flink-avro", "target", "avro*.jar")
+    if len(glob.glob(avro_jar_pattern)) > 0:
+        # the avro jar already existed, just return.
+        return
+    mvn = "mvn.cmd" if on_windows() else "mvn"
+    avro_version_output = check_output(
+        [mvn, "help:evaluate", "-Dexpression=avro.version"],
+        cwd=flink_source_root).decode("utf-8")
+    lines = avro_version_output.replace("\r", "").split("\n")
+    avro_version = None
+    for line in lines:
+        if not line.startswith("[") and not line.startswith("Download") and line.strip() != "":
+            avro_version = line
+            break
+    if avro_version is None:
+        raise Exception("The Apache Avro version is not found in the maven command output:\n %s" %
+                        avro_version_output)
+    print(check_output(

Review comment:
       remove the print?

##########
File path: flink-python/src/main/resources/pyflink-udf-runner.sh
##########
@@ -40,4 +40,4 @@ if [[ "$_PYTHON_WORKING_DIR" != "" ]]; then
 fi
 
 log="$BOOT_LOG_DIR/flink-python-udf-boot.log"
-${python} -m pyflink.fn_execution.boot $@ 2>&1 | tee -a ${log}

Review comment:
       what's the purpose of this change

##########
File path: flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java
##########
@@ -288,6 +307,10 @@ private void constructRequirementsDirectory(Map<String, String> env) throws IOEx
 		}
 	}
 
+	private String getKillWorkerScriptPath() {

Review comment:
       this method is not used

##########
File path: flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java
##########
@@ -127,20 +130,36 @@ public void open() throws Exception {
 	}
 
 	@Override
-	public void close() {
-		FileUtils.deleteDirectoryQuietly(new File(baseDirectory));
-		if (shutdownHook != null) {
-			ShutdownHookUtil.removeShutdownHook(
-				shutdownHook, ProcessPythonEnvironmentManager.class.getSimpleName(), LOG);
-			shutdownHook = null;
+	public void close() throws Exception {
+		try {
+			int i = 0;
+			while (i < CHECK_TIMEOUT / CHECK_INTERVAL) {

Review comment:
       ```suggestion
   			int retries = 0;
   			while (true) {
   			    try {
   			       FileUtils.deleteDirectory(new File(baseDirectory));
   			       break;
   			    } catch (Throwable t) {
   			       retries++;
   			       if (retries <= CHECK_TIMEOUT / CHECK_INTERVAL) {
   			         LOG.warning("Failed to delete the working directory {} of the Python UDF worker. Retrying...", baseDirectory, t);
   			       } else {
   			          LOG.warning("Failed to delete the working directory {} of the Python UDF worker.", baseDirectory, t);
   			          break;
   			       }
   			    }
   			}
   ```

##########
File path: flink-python/src/main/java/org/apache/flink/client/python/PythonGatewayServer.java
##########
@@ -71,15 +75,42 @@ public static void main(String[] args) throws IOException, ExecutionException, I
 		}
 
 		try {
-			// Exit on EOF or broken pipe.  This ensures that the server dies
-			// if its parent program dies.
-			while (System.in.read() != -1) {

Review comment:
       I'm not sure why this doesn't work in windows?

##########
File path: flink-python/pyflink/table/tests/test_dependency.py
##########
@@ -185,7 +185,10 @@ def test_set_environment(self):
         python_exec = sys.executable
         tmp_dir = self.tempdir
         python_exec_link_path = os.path.join(tmp_dir, "py_exec")
-        os.symlink(python_exec, python_exec_link_path)
+        try:

Review comment:
       use `unittest.skipIf(onWindows)`.

##########
File path: flink-python/pyflink/python_callback_server.py
##########
@@ -15,7 +15,6 @@
 #  See the License for the specific language governing permissions and
 # limitations under the License.

Review comment:
       does it make sense to rename this script to `pyflink_callback_server.py`?

##########
File path: flink-python/pyflink/datastream/tests/test_state_backend.py
##########
@@ -132,7 +133,8 @@ def test_get_checkpoint_backend(self):
         self.assertEqual(checkpoint_backend.get_checkpoint_path(), "file://var/checkpoints")
 
     def test_get_set_db_storage_paths(self):
-
+        if on_windows():

Review comment:
       could we improve this test a bit to make it runnable on windows?

##########
File path: flink-python/pyflink/fn_execution/tests/test_process_mode_boot.py
##########
@@ -114,7 +117,7 @@ def run_boot_py(self):
                 "--control_endpoint", "localhost:0000",
                 "--semi_persist_dir", self.tmp_dir]
 
-        return subprocess.call(args, stdout=sys.stdout, stderr=sys.stderr, env=self.env)

Review comment:
       Why change this?

##########
File path: flink-python/pyflink/table/tests/test_pandas_udf.py
##########
@@ -142,7 +142,7 @@ def time_func(time_param):
                 'time_param of wrong type %s !' % type(time_param[0])
             return time_param
 
-        timestamp_value = datetime.datetime(1970, 1, 1, 0, 0, 0, 123000)
+        timestamp_value = datetime.datetime(1970, 1, 2, 0, 0, 0, 123000)

Review comment:
       Why change this

##########
File path: flink-python/pyflink/pyflink_gateway_server.py
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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.
+################################################################################
+import argparse
+import getpass
+import glob
+import os
+import platform
+import re
+import signal
+import socket
+import sys
+from collections import namedtuple
+from string import Template
+from subprocess import Popen, PIPE, check_output
+
+from pyflink.find_flink_home import _find_flink_home, _find_flink_source_root
+
+
+def on_windows():
+    return platform.system() == "Windows"
+
+
+def find_java_executable():
+    java_executable = "java.exe" if on_windows() else "java"
+    flink_home = _find_flink_home()
+    flink_conf_path = os.path.join(flink_home, "conf", "flink-conf.yaml")
+    java_home = None
+
+    if os.path.isfile(flink_conf_path):
+        with open(flink_conf_path, "r") as f:
+            flink_conf_yaml = f.read()
+        java_homes = re.findall(r'^[ ]*env\.java\.home[ ]*: ([^#]*).*$', flink_conf_yaml)
+        if len(java_homes) > 1:
+            java_home = java_homes[len(java_homes) - 1].strip()
+
+    if java_home is None and "JAVA_HOME" in os.environ:
+        java_home = os.environ["JAVA_HOME"]
+
+    if java_home is not None:
+        java_executable = os.path.join(java_home, "bin", java_executable)
+
+    return java_executable
+
+
+def construct_log_settings():
+    templates = [
+        "-Dlog.file=${flink_log_dir}/flink-${flink_ident_string}-python-${hostname}.log",
+        "-Dlog4j.configuration=${flink_conf_dir}/log4j-cli.properties",
+        "-Dlog4j.configurationFile=${flink_conf_dir}/log4j-cli.properties",
+        "-Dlogback.configurationFile=${flink_conf_dir}/logback.xml"
+    ]
+
+    flink_home = _find_flink_home()
+    flink_conf_dir = os.path.join(flink_home, "conf")
+    flink_log_dir = os.path.join(flink_home, "log")
+    if "FLINK_IDENT_STRING" in os.environ:
+        flink_ident_string = os.environ["FLINK_IDENT_STRING"]
+    else:
+        flink_ident_string = getpass.getuser()
+    hostname = socket.gethostname()
+    log_settings = []
+    for template in templates:
+        log_settings.append(Template(template).substitute(
+            flink_conf_dir=flink_conf_dir,
+            flink_log_dir=flink_log_dir,
+            flink_ident_string=flink_ident_string,
+            hostname=hostname))
+    return log_settings
+
+
+def construct_classpath():
+    flink_home = _find_flink_home()
+    if on_windows():
+        # The command length is limited on Windows. To avoid the problem we should shorten the
+        # command length as much as possible.
+        lib_jars = os.path.join(flink_home, "lib", "*")
+    else:
+        lib_jars = os.pathsep.join(glob.glob(os.path.join(flink_home, "lib", "*.jar")))
+
+    flink_python_jars = glob.glob(os.path.join(flink_home, "opt", "flink-python*.jar"))
+    if len(flink_python_jars) < 1:
+        print("The flink-python jar is not found in the opt folder of the FLINK_HOME: %s" %
+              flink_home)
+        return lib_jars
+    flink_python_jar = flink_python_jars[0]
+
+    return os.pathsep.join([lib_jars, flink_python_jar])
+
+
+def download_apache_avro():
+    """
+    Currently we need to download the Apache Avro manually to avoid test failure caused by the avro
+    format sql jar. See https://issues.apache.org/jira/browse/FLINK-17417. If the issue is fixed,
+    this method could be removed. Using maven command copy the jars in repository to avoid accessing
+    external network.
+    """
+    flink_source_root = _find_flink_source_root()
+    avro_jar_pattern = os.path.join(
+        flink_source_root, "flink-formats", "flink-avro", "target", "avro*.jar")
+    if len(glob.glob(avro_jar_pattern)) > 0:
+        # the avro jar already existed, just return.
+        return
+    mvn = "mvn.cmd" if on_windows() else "mvn"
+    avro_version_output = check_output(
+        [mvn, "help:evaluate", "-Dexpression=avro.version"],
+        cwd=flink_source_root).decode("utf-8")
+    lines = avro_version_output.replace("\r", "").split("\n")
+    avro_version = None
+    for line in lines:
+        if not line.startswith("[") and not line.startswith("Download") and line.strip() != "":
+            avro_version = line
+            break
+    if avro_version is None:
+        raise Exception("The Apache Avro version is not found in the maven command output:\n %s" %
+                        avro_version_output)
+    print(check_output(
+        [mvn,
+         "org.apache.maven.plugins:maven-dependency-plugin:2.10:copy",
+         "-Dartifact=org.apache.avro:avro:%s:jar" % avro_version,
+         "-DoutputDirectory=%s/flink-formats/flink-avro/target" % flink_source_root],
+        cwd=flink_source_root))
+
+
+def construct_test_classpath():
+    test_jar_patterns = [
+        "flink-table/flink-table-planner/target/flink-table-planner*-tests.jar",
+        "flink-runtime/target/flink-runtime*tests.jar",
+        "flink-streaming-java/target/flink-streaming-java*tests.jar",
+        "flink-formats/flink-csv/target/flink-csv*.jar",
+        "flink-formats/flink-avro/target/flink-avro*.jar",
+        "flink-formats/flink-avro/target/avro*.jar",
+        "flink-formats/flink-json/target/flink-json*.jar",
+        "flink-connectors/flink-connector-elasticsearch-base/target/flink*.jar",
+        "flink-connectors/flink-connector-kafka-base/target/flink*.jar",
+        "flink-ml-parent/flink-ml-api/target/flink-ml-api*.jar",
+        "flink-ml-parent/flink-ml-lib/target/flink-ml-lib*.jar",
+    ]
+    test_jars = []
+    flink_source_root = _find_flink_source_root()
+    for pattern in test_jar_patterns:
+        pattern = pattern.replace("/", os.path.sep)
+        test_jars += glob.glob(os.path.join(flink_source_root, pattern))
+    return os.path.pathsep.join(test_jars)
+
+
+def construct_program_args(args):
+    parser = argparse.ArgumentParser()
+    parser.add_argument("-c", "--class", required=True)
+    parser.add_argument("cluster_type", choices=["local", "remote", "yarn"])
+    parse_result, other_args = parser.parse_known_args(args)
+    main_class = getattr(parse_result, "class")
+    cluster_type = parse_result.cluster_type
+    return namedtuple(
+        "ProgramArgs", ["main_class", "cluster_type", "other_args"])(
+        main_class, cluster_type, other_args)
+
+
+def prepare_environment_variable(env):
+    flink_home = _find_flink_home()
+    env = dict(env)
+    env["FLINK_CONF_DIR"] = os.path.join(flink_home, "conf")
+    env["FLINK_BIN_DIR"] = os.path.join(flink_home, "bin")
+    env["FLINK_PLUGINS_DIR"] = os.path.join(flink_home, "plugins")
+    env["FLINK_LIB_DIR"] = os.path.join(flink_home, "lib")
+    env["FLINK_OPT_DIR"] = os.path.join(flink_home, "opt")
+    return env
+
+
+def launch_gateway_server_process(env, args):
+    java_executable = find_java_executable()
+    log_settings = construct_log_settings()
+    classpath = construct_classpath()
+    env = prepare_environment_variable(env)
+    if "FLINK_TESTING" in env:
+        download_apache_avro()
+        classpath = os.pathsep.join([classpath, construct_test_classpath()])
+    program_args = construct_program_args(args)
+    if program_args.cluster_type == "local":
+        command = [java_executable] + log_settings + ["-cp", classpath, program_args.main_class] \
+            + program_args.other_args
+    else:
+        command = [os.path.join(env["FLINK_BIN_DIR"], "flink"), "run"] + program_args.other_args \
+            + ["-c", program_args.main_class]
+    preexec_fn = None
+    if not on_windows():
+        def preexec_func():
+            # ignore ctrl-c / SIGINT

Review comment:
       why only need to ignore `ctrl-c / SIGINT` in linux?




----------------------------------------------------------------
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:
users@infra.apache.org