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



##########
File path: flink-python/pyflink/datastream/stream_execution_environment.py
##########
@@ -553,6 +553,45 @@ def set_python_executable(self, python_exec: str):
             .getEnvironmentConfig(self._j_stream_execution_environment)
         env_config.setString(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), 
python_exec)
 
+    def add_jars(self, *jars_path: List[str]):

Review comment:
       I think the signature should be:
   ```suggestion
       def add_jars(self, *jars_path: str):
   ```
   
   What do you think?

##########
File path: flink-python/pyflink/datastream/stream_execution_environment.py
##########
@@ -553,6 +553,45 @@ def set_python_executable(self, python_exec: str):
             .getEnvironmentConfig(self._j_stream_execution_environment)
         env_config.setString(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), 
python_exec)
 
+    def add_jars(self, *jars_path: List[str]):
+        """
+        Adds a list of jar files that will be uploaded to the cluster and 
referenced by the job.
+
+        :param jars_path: Path of jars that delimited by ';'.
+        """
+        add_jars_to_context_class_loader(*jars_path)
+        jvm = get_gateway().jvm
+        jars_key = 
jvm.org.apache.flink.configuration.PipelineOptions.JARS.key()
+        env_config = jvm.org.apache.flink.python.util.PythonConfigUtil \
+            .getEnvironmentConfig(self._j_stream_execution_environment)
+        set_jar_paths = env_config.getString(jars_key, None)
+        if set_jar_paths is None:
+            set_jar_paths = ''
+        jars_path = jvm.PythonDependencyUtils.FILE_DELIMITER.join(*jars_path)
+        python_files = 
jvm.PythonDependencyUtils.FILE_DELIMITER.join([set_jar_paths,

Review comment:
       Why named python_files?

##########
File path: flink-python/pyflink/datastream/stream_execution_environment.py
##########
@@ -553,6 +553,45 @@ def set_python_executable(self, python_exec: str):
             .getEnvironmentConfig(self._j_stream_execution_environment)
         env_config.setString(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), 
python_exec)
 
+    def add_jars(self, *jars_path: List[str]):
+        """
+        Adds a list of jar files that will be uploaded to the cluster and 
referenced by the job.
+
+        :param jars_path: Path of jars that delimited by ';'.
+        """
+        add_jars_to_context_class_loader(*jars_path)
+        jvm = get_gateway().jvm
+        jars_key = 
jvm.org.apache.flink.configuration.PipelineOptions.JARS.key()
+        env_config = jvm.org.apache.flink.python.util.PythonConfigUtil \
+            .getEnvironmentConfig(self._j_stream_execution_environment)
+        set_jar_paths = env_config.getString(jars_key, None)

Review comment:
       Why not as following?
   ```suggestion
           set_jar_paths = env_config.getString(jars_key, '')
   ```

##########
File path: flink-python/pyflink/datastream/stream_execution_environment.py
##########
@@ -553,6 +553,45 @@ def set_python_executable(self, python_exec: str):
             .getEnvironmentConfig(self._j_stream_execution_environment)
         env_config.setString(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), 
python_exec)
 
+    def add_jars(self, *jars_path: List[str]):
+        """
+        Adds a list of jar files that will be uploaded to the cluster and 
referenced by the job.
+
+        :param jars_path: Path of jars that delimited by ';'.

Review comment:
       I think the comment isn't correct and should also be updated.




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


Reply via email to