You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by hx...@apache.org on 2022/03/29 03:05:26 UTC

[flink] branch release-1.15 updated: [FLINK-26421][python] Remove TableConfig from StreamTableEnvironment#create

This is an automated email from the ASF dual-hosted git repository.

hxb pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.15 by this push:
     new 372b82c  [FLINK-26421][python] Remove TableConfig from StreamTableEnvironment#create
372b82c is described below

commit 372b82c841906c5b7f3808e9206a3a34d172d4b8
Author: huangxingbo <hx...@gmail.com>
AuthorDate: Tue Mar 29 11:00:28 2022 +0800

    [FLINK-26421][python] Remove TableConfig from StreamTableEnvironment#create
    
    Following: 57742b85095147711070c566069244c40ed8e77c remove the
    `TableConfig` from `StreamTableEnviroment#create()` and allow
    configuration only via `EnviromentSettings.with_configuration()`.
---
 flink-python/pyflink/table/table_environment.py | 29 +++----------------------
 1 file changed, 3 insertions(+), 26 deletions(-)

diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py
index 6e73799..e001f36 100644
--- a/flink-python/pyflink/table/table_environment.py
+++ b/flink-python/pyflink/table/table_environment.py
@@ -1653,7 +1653,6 @@ class StreamTableEnvironment(TableEnvironment):
 
     @staticmethod
     def create(stream_execution_environment: StreamExecutionEnvironment = None,  # type: ignore
-               table_config: TableConfig = None,
                environment_settings: EnvironmentSettings = None) -> 'StreamTableEnvironment':
         """
         Creates a :class:`~pyflink.table.StreamTableEnvironment`.
@@ -1664,10 +1663,6 @@ class StreamTableEnvironment(TableEnvironment):
             # create with StreamExecutionEnvironment.
             >>> env = StreamExecutionEnvironment.get_execution_environment()
             >>> table_env = StreamTableEnvironment.create(env)
-            # create with StreamExecutionEnvironment and TableConfig.
-            >>> table_config = TableConfig()
-            >>> table_config.set_null_check(False)
-            >>> table_env = StreamTableEnvironment.create(env, table_config)
             # create with StreamExecutionEnvironment and EnvironmentSettings.
             >>> configuration = Configuration()
             >>> configuration.set_string('execution.buffer-timeout', '1 min')
@@ -1685,27 +1680,15 @@ class StreamTableEnvironment(TableEnvironment):
         :param stream_execution_environment: The
                                              :class:`~pyflink.datastream.StreamExecutionEnvironment`
                                              of the TableEnvironment.
-        :param table_config: The configuration of the TableEnvironment, optional.
         :param environment_settings: The environment settings used to instantiate the
                                      TableEnvironment.
         :return: The StreamTableEnvironment created from given StreamExecutionEnvironment and
                  configuration.
         """
         if stream_execution_environment is None and \
-                table_config is None and \
                 environment_settings is None:
             raise ValueError("No argument found, the param 'stream_execution_environment' "
                              "or 'environment_settings' is required.")
-        elif stream_execution_environment is None and \
-                table_config is not None and \
-                environment_settings is None:
-            raise ValueError("Only the param 'table_config' is found, "
-                             "the param 'stream_execution_environment' is also required.")
-        if table_config is not None and \
-                environment_settings is not None:
-            raise ValueError("The param 'table_config' and "
-                             "'environment_settings' cannot be used at the same time")
-
         gateway = get_gateway()
         if environment_settings is not None:
             if stream_execution_environment is None:
@@ -1716,15 +1699,9 @@ class StreamTableEnvironment(TableEnvironment):
                     stream_execution_environment._j_stream_execution_environment,
                     environment_settings._j_environment_settings)
         else:
-            if table_config is not None:
-                warnings.warn("Deprecated in 1.15, please use EnvironmentSettings.",
-                              DeprecationWarning)
-                j_tenv = gateway.jvm.StreamTableEnvironment.create(
-                    stream_execution_environment._j_stream_execution_environment,
-                    table_config._j_table_config)
-            else:
-                j_tenv = gateway.jvm.StreamTableEnvironment.create(
-                    stream_execution_environment._j_stream_execution_environment)
+            j_tenv = gateway.jvm.StreamTableEnvironment.create(
+                stream_execution_environment._j_stream_execution_environment)
+
         return StreamTableEnvironment(j_tenv)
 
     def from_data_stream(self,