wuchong commented on a change in pull request #15006:
URL: https://github.com/apache/flink/pull/15006#discussion_r589163832



##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/DefaultContext.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.client.cli.CliArgsException;
+import org.apache.flink.client.cli.CliFrontendParser;
+import org.apache.flink.client.cli.CustomCommandLine;
+import org.apache.flink.client.cli.ExecutionConfigAccessor;
+import org.apache.flink.client.cli.ProgramOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.RestartStrategyOptions;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.plugin.PluginUtils;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamPipelineOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.config.entries.DeploymentEntry;
+import org.apache.flink.table.client.config.entries.ExecutionEntry;
+import org.apache.flink.table.client.gateway.Executor;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.descriptors.FunctionDescriptorValidator;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Context describing default environment, command line options, flink config, 
etc.
+ *
+ * <p>When the {@link Executor} execute `reset` commands, the session can 
restore from the "default"
+ * context.
+ */
+public class DefaultContext {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(DefaultContext.class);
+
+    private final Environment defaultEnv;
+    private final List<URL> dependencies;
+    private final Configuration flinkConfig;
+
+    public DefaultContext(
+            Environment defaultEnv,
+            List<URL> dependencies,
+            Configuration flinkConfig,
+            List<CustomCommandLine> commandLines) {
+        this.defaultEnv = defaultEnv;
+        this.dependencies = dependencies;
+        this.flinkConfig = flinkConfig;
+        Options commandLineOptions = collectCommandLineOptions(commandLines);
+
+        // initialize default file system
+        FileSystem.initialize(
+                flinkConfig, 
PluginUtils.createPluginManagerFromRootFolder(flinkConfig));
+
+        // add python dependencies
+        if (containsPythonFunction(defaultEnv)) {
+            addPythonDependency();
+        }
+
+        // put environment entry into Configuration
+        initConfigurationFromConfigurationEntry();

Review comment:
       `initConfigurationFromEnvironment();`

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
##########
@@ -75,7 +75,15 @@
     private MathContext decimalContext = MathContext.DECIMAL128;
 
     /** A configuration object to hold all key/value configuration. */
-    private Configuration configuration = new Configuration();
+    private Configuration configuration;

Review comment:
       revert changes to this file. 

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/LegacyTableEnvironmentInitializer.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.bridge.java.BatchTableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.config.entries.SinkTableEntry;
+import org.apache.flink.table.client.config.entries.SourceSinkTableEntry;
+import org.apache.flink.table.client.config.entries.SourceTableEntry;
+import org.apache.flink.table.client.config.entries.TemporalTableEntry;
+import org.apache.flink.table.client.config.entries.ViewEntry;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.descriptors.CoreModuleDescriptorValidator;
+import org.apache.flink.table.factories.BatchTableSinkFactory;
+import org.apache.flink.table.factories.BatchTableSourceFactory;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.flink.table.factories.ModuleFactory;
+import org.apache.flink.table.factories.TableFactoryService;
+import org.apache.flink.table.factories.TableSinkFactory;
+import org.apache.flink.table.factories.TableSinkFactoryContextImpl;
+import org.apache.flink.table.factories.TableSourceFactory;
+import org.apache.flink.table.factories.TableSourceFactoryContextImpl;
+import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.functions.FunctionService;
+import org.apache.flink.table.functions.ScalarFunction;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.module.Module;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import java.net.URLClassLoader;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.flink.table.api.Expressions.$;
+
+/** Utils to initialize {@link TableEnvironment} from {@link Environment}. */
+public class LegacyTableEnvironmentInitializer {

Review comment:
       Add a comment that this file can be removed once we don't support 
`sql-client.yaml`.

##########
File path: 
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.cli.DefaultCLI;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.utils.EnvironmentFileUtil;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_SQL_DIALECT;
+
+/** Test {@link SessionContext}. */
+public class SessionContextTest {
+
+    private static final String DEFAULTS_ENVIRONMENT_FILE = 
"test-sql-client-defaults.yaml";
+
+    @Test
+    public void testSetAndResetYamlKey() throws Exception {
+        SessionContext sessionContext = createSessionContext();
+        sessionContext.set("execution.max-table-result-rows", "100000");
+        Assert.assertEquals(
+                "100000",
+                sessionContext
+                        .getSessionEnvironment()
+                        .getExecution()
+                        .asMap()
+                        .get("max-table-result-rows"));
+
+        sessionContext.reset();
+
+        Assert.assertEquals(
+                "100",

Review comment:
       What's the influence scope of `reset` ? Should it reset all 
configuration to default value, or rollback to initial value defined in 
`sql-client-defaults.yaml` or `init.sql`? 
   
   If it reset to default value, then the expected value of 
`max-table-result-rows` should be `1000000`.

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.Executor;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * Context describing a session, it's mainly used for user to open a new 
session in the backend. If
+ * client request to open a new session, the backend {@link Executor} will 
maintain the session
+ * context map util users close it.
+ */
+public class SessionContext {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SessionContext.class);
+
+    private final String sessionId;
+    private final DefaultContext defaultContext;
+
+    private Environment sessionEnv;
+    private final Configuration sessionConfiguration;
+
+    private final SessionState sessionState;
+    private final URLClassLoader classLoader;
+    private ExecutionContext executionContext;
+
+    public SessionContext(
+            DefaultContext defaultContext,
+            String sessionId,
+            Environment sessionEnv,
+            Configuration sessionConfiguration,
+            URLClassLoader classLoader,
+            SessionState sessionState,
+            ExecutionContext executionContext) {
+        this.defaultContext = defaultContext;
+        this.sessionId = sessionId;
+        this.sessionEnv = sessionEnv;
+        this.sessionConfiguration = sessionConfiguration;
+        this.classLoader = classLoader;
+        this.sessionState = sessionState;
+        this.executionContext = executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Getter method
+    // 
--------------------------------------------------------------------------------------------
+
+    public String getSessionId() {
+        return this.sessionId;
+    }
+
+    public Environment getSessionEnvironment() {
+        return this.sessionEnv;
+    }
+
+    public ExecutionContext getExecutionContext() {
+        return this.executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Method to execute commands
+    // 
--------------------------------------------------------------------------------------------
+
+    /** Reset properties to default. It will rebuild a new {@link 
ExecutionContext}. */
+    public void reset() {
+        sessionEnv = defaultContext.getDefaultEnv().clone();
+        for (String key : sessionConfiguration.toMap().keySet()) {
+            // Don't care the type of the option
+            ConfigOption<String> keyToDelete = 
ConfigOptions.key(key).stringType().noDefaultValue();
+            sessionConfiguration.removeConfig(keyToDelete);
+        }
+        sessionConfiguration.addAll(defaultContext.getFlinkConfig());

Review comment:
       What about changing `sessionConfiguration` to not-final? We can create a 
new configuration for the session here, `new 
Configuration(defaultContext.getFlinkConfig())`. This would be much more 
straightforward. I think it's fine to declare not-final here, as other session 
varaible (e.g. `executionContext`) are also not-final. 

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableException;
+import 
org.apache.flink.table.api.bridge.java.internal.BatchTableEnvironmentImpl;
+import 
org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.delegation.Executor;
+import org.apache.flink.table.delegation.ExecutorFactory;
+import org.apache.flink.table.delegation.Planner;
+import org.apache.flink.table.delegation.PlannerFactory;
+import org.apache.flink.table.factories.ComponentFactoryService;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import java.lang.reflect.Method;
+import java.net.URLClassLoader;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+
+import static 
org.apache.flink.table.client.gateway.context.SessionContext.SessionState;
+
+/**
+ * Context for executing table programs. This class caches everything that can 
be cached across
+ * multiple queries as long as the session context does not change. This must 
be thread-safe as it
+ * might be reused across different query submissions.
+ */
+public class ExecutionContext {

Review comment:
       I'm wondering `ExecutionContext` is also not needed anymore. The only 
thing in `ExecutionContext` been used by other components is the `classloader` 
and the `tableEnv`. However, IMO, `classloader` is a more session variable 
which should be get from `SessionContext` (e.g. we may support `ADD JAR` to add 
jar dependency to the session in the future).  Then `ExecutionContext` would be 
just a wrapper of `TableEnvironment`. 
   
   If this is the case, what do you think about createing a new 
`TableEnvironment` instead of `ExecutionContext` in `SessionContext`.?

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.Executor;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * Context describing a session, it's mainly used for user to open a new 
session in the backend. If
+ * client request to open a new session, the backend {@link Executor} will 
maintain the session
+ * context map util users close it.
+ */
+public class SessionContext {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SessionContext.class);
+
+    private final String sessionId;
+    private final DefaultContext defaultContext;
+
+    private Environment sessionEnv;
+    private final Configuration sessionConfiguration;
+
+    private final SessionState sessionState;
+    private final URLClassLoader classLoader;
+    private ExecutionContext executionContext;
+
+    public SessionContext(

Review comment:
       We can make it `private`, so users can all use `create` method for 
initialization. 

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.Executor;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * Context describing a session, it's mainly used for user to open a new 
session in the backend. If
+ * client request to open a new session, the backend {@link Executor} will 
maintain the session
+ * context map util users close it.
+ */
+public class SessionContext {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SessionContext.class);
+
+    private final String sessionId;
+    private final DefaultContext defaultContext;
+
+    private Environment sessionEnv;
+    private final Configuration sessionConfiguration;
+
+    private final SessionState sessionState;
+    private final URLClassLoader classLoader;
+    private ExecutionContext executionContext;
+
+    public SessionContext(
+            DefaultContext defaultContext,
+            String sessionId,
+            Environment sessionEnv,
+            Configuration sessionConfiguration,
+            URLClassLoader classLoader,
+            SessionState sessionState,
+            ExecutionContext executionContext) {
+        this.defaultContext = defaultContext;
+        this.sessionId = sessionId;
+        this.sessionEnv = sessionEnv;
+        this.sessionConfiguration = sessionConfiguration;
+        this.classLoader = classLoader;
+        this.sessionState = sessionState;
+        this.executionContext = executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Getter method
+    // 
--------------------------------------------------------------------------------------------
+
+    public String getSessionId() {
+        return this.sessionId;
+    }
+
+    public Environment getSessionEnvironment() {
+        return this.sessionEnv;
+    }
+
+    public ExecutionContext getExecutionContext() {
+        return this.executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Method to execute commands
+    // 
--------------------------------------------------------------------------------------------
+
+    /** Reset properties to default. It will rebuild a new {@link 
ExecutionContext}. */
+    public void reset() {
+        sessionEnv = defaultContext.getDefaultEnv().clone();
+        for (String key : sessionConfiguration.toMap().keySet()) {
+            // Don't care the type of the option
+            ConfigOption<String> keyToDelete = 
ConfigOptions.key(key).stringType().noDefaultValue();
+            sessionConfiguration.removeConfig(keyToDelete);
+        }
+        sessionConfiguration.addAll(defaultContext.getFlinkConfig());
+        executionContext = new ExecutionContext(sessionEnv, executionContext);
+    }
+
+    /** Set properties. It will rebuild a new {@link ExecutionContext} */
+    public void set(String key, String value) {
+        // put key-value into the Environment
+        try {
+            this.sessionEnv = Environment.enrich(sessionEnv, 
Collections.singletonMap(key, value));
+            sessionConfiguration.setString(key, value);
+        } catch (Throwable t) {
+            throw new SqlExecutionException("Could not set session property.", 
t);
+        }
+
+        // Renew the ExecutionContext by new environment.
+        // Book keep all the session states of current ExecutionContext then
+        // re-register them into the new one.
+        this.executionContext = new ExecutionContext(sessionEnv, 
executionContext);
+    }
+
+    /** Close resources, e.g. catalogs. */
+    public void close() {
+        try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(classLoader)) {
+            for (String name : sessionState.catalogManager.listCatalogs()) {
+                
sessionState.catalogManager.getCatalog(name).ifPresent(Catalog::close);
+            }
+        }
+        try {
+            classLoader.close();
+        } catch (IOException e) {
+            LOG.debug("Error while closing class loader.", e);
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Helper method to create
+    // 
--------------------------------------------------------------------------------------------
+
+    public static SessionContext create(DefaultContext defaultContext, String 
sessionId) {
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init config
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        Environment sessionEnv = defaultContext.getDefaultEnv().clone();
+        Configuration configuration = defaultContext.getFlinkConfig().clone();
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init classloader
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        URLClassLoader classLoader =
+                ClientUtils.buildUserCodeClassLoader(
+                        defaultContext.getDependencies(),
+                        Collections.emptyList(),
+                        SessionContext.class.getClassLoader(),
+                        configuration);
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init session state
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        ModuleManager moduleManager = new ModuleManager();
+
+        final EnvironmentSettings settings = 
sessionEnv.getExecution().getEnvironmentSettings();
+
+        CatalogManager catalogManager =
+                CatalogManager.newBuilder()
+                        .classLoader(classLoader)
+                        .config(configuration)
+                        .defaultCatalog(
+                                settings.getBuiltInCatalogName(),
+                                new GenericInMemoryCatalog(
+                                        settings.getBuiltInCatalogName(),
+                                        settings.getBuiltInDatabaseName()))
+                        .build();
+
+        FunctionCatalog functionCatalog =
+                new FunctionCatalog(configuration, catalogManager, 
moduleManager);
+        SessionState sessionState =
+                new SessionState(catalogManager, moduleManager, 
functionCatalog);
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        return new SessionContext(
+                defaultContext,
+                sessionId,
+                sessionEnv,
+                configuration,
+                classLoader,
+                sessionState,
+                new ExecutionContext(sessionEnv, configuration, classLoader, 
sessionState));
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof SessionContext)) {
+            return false;
+        }
+        SessionContext context = (SessionContext) o;
+        return Objects.equals(sessionId, context.sessionId)
+                && Objects.equals(defaultContext, context.defaultContext)
+                && Objects.equals(sessionEnv, context.sessionEnv)
+                && Objects.equals(sessionConfiguration, 
context.sessionConfiguration)
+                && Objects.equals(classLoader, context.classLoader)
+                && Objects.equals(sessionState, context.sessionState)
+                && Objects.equals(executionContext, context.executionContext);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                sessionId,
+                defaultContext,
+                sessionEnv,
+                sessionConfiguration,
+                classLoader,
+                sessionState,
+                executionContext);
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Inner class
+    // 
--------------------------------------------------------------------------------------------
+
+    /** session state. */
+    public static class SessionState {
+
+        public final CatalogManager catalogManager;
+        public final FunctionCatalog functionCatalog;
+        public final ModuleManager moduleManager;
+
+        public SessionState(
+                CatalogManager catalogManager,
+                ModuleManager moduleManager,
+                FunctionCatalog functionCatalog) {
+            this.catalogManager = catalogManager;
+            this.moduleManager = moduleManager;
+            this.functionCatalog = functionCatalog;
+        }
+
+        @Override
+        public boolean equals(Object obj) {

Review comment:
       Do we need this?

##########
File path: 
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.cli.DefaultCLI;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.utils.EnvironmentFileUtil;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_SQL_DIALECT;
+
+/** Test {@link SessionContext}. */
+public class SessionContextTest {
+
+    private static final String DEFAULTS_ENVIRONMENT_FILE = 
"test-sql-client-defaults.yaml";
+
+    @Test
+    public void testSetAndResetYamlKey() throws Exception {
+        SessionContext sessionContext = createSessionContext();
+        sessionContext.set("execution.max-table-result-rows", "100000");
+        Assert.assertEquals(
+                "100000",
+                sessionContext
+                        .getSessionEnvironment()
+                        .getExecution()
+                        .asMap()
+                        .get("max-table-result-rows"));
+
+        sessionContext.reset();
+
+        Assert.assertEquals(
+                "100",
+                sessionContext
+                        .getSessionEnvironment()
+                        .getExecution()
+                        .asMap()
+                        .get("max-table-result-rows"));
+    }
+
+    @Test
+    public void testSetAndResetOption() throws Exception {

Review comment:
       Could you try to set and reset `CoreOptions#DEFAULT_PARALLELISM`?

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java
##########
@@ -91,17 +102,17 @@ private void start() {
             } else {
                 libDirs = Collections.emptyList();
             }
-            final Executor executor = new LocalExecutor(options.getDefaults(), 
jars, libDirs);
+            final Executor executor = new LocalExecutor();
             executor.start();
 
-            // create CLI client with session environment
-            final Environment sessionEnv = 
readSessionEnvironment(options.getEnvironment());
-            appendPythonConfig(sessionEnv, options.getPythonConfiguration());
+            // build default context
+            DefaultContext defaultContext = buildEmbeddedDefaultContext(jars, 
libDirs);

Review comment:
       Personally, I think `buildEmbeddedDefaultContext` should belong to 
`LocalExecutor`. `SqlClient` shouldn't care about how to build context, it just 
need to setup Executor (`LocalExecutor`, or `GatewayExecutor` in the future). 
   
   I think we can also improve the interface of `Executor`, changing 
`openSession` to `openSession(String sessionId)` and create `SessionContext` in 
it. 

##########
File path: 
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.table.client.gateway.context;
+
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.client.config.Environment;
+import org.apache.flink.table.client.gateway.Executor;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.util.TemporaryClassLoaderContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * Context describing a session, it's mainly used for user to open a new 
session in the backend. If
+ * client request to open a new session, the backend {@link Executor} will 
maintain the session
+ * context map util users close it.
+ */
+public class SessionContext {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SessionContext.class);
+
+    private final String sessionId;
+    private final DefaultContext defaultContext;
+
+    private Environment sessionEnv;
+    private final Configuration sessionConfiguration;
+
+    private final SessionState sessionState;
+    private final URLClassLoader classLoader;
+    private ExecutionContext executionContext;
+
+    public SessionContext(
+            DefaultContext defaultContext,
+            String sessionId,
+            Environment sessionEnv,
+            Configuration sessionConfiguration,
+            URLClassLoader classLoader,
+            SessionState sessionState,
+            ExecutionContext executionContext) {
+        this.defaultContext = defaultContext;
+        this.sessionId = sessionId;
+        this.sessionEnv = sessionEnv;
+        this.sessionConfiguration = sessionConfiguration;
+        this.classLoader = classLoader;
+        this.sessionState = sessionState;
+        this.executionContext = executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Getter method
+    // 
--------------------------------------------------------------------------------------------
+
+    public String getSessionId() {
+        return this.sessionId;
+    }
+
+    public Environment getSessionEnvironment() {
+        return this.sessionEnv;
+    }
+
+    public ExecutionContext getExecutionContext() {
+        return this.executionContext;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Method to execute commands
+    // 
--------------------------------------------------------------------------------------------
+
+    /** Reset properties to default. It will rebuild a new {@link 
ExecutionContext}. */
+    public void reset() {
+        sessionEnv = defaultContext.getDefaultEnv().clone();
+        for (String key : sessionConfiguration.toMap().keySet()) {
+            // Don't care the type of the option
+            ConfigOption<String> keyToDelete = 
ConfigOptions.key(key).stringType().noDefaultValue();
+            sessionConfiguration.removeConfig(keyToDelete);
+        }
+        sessionConfiguration.addAll(defaultContext.getFlinkConfig());
+        executionContext = new ExecutionContext(sessionEnv, executionContext);
+    }
+
+    /** Set properties. It will rebuild a new {@link ExecutionContext} */
+    public void set(String key, String value) {
+        // put key-value into the Environment
+        try {
+            this.sessionEnv = Environment.enrich(sessionEnv, 
Collections.singletonMap(key, value));
+            sessionConfiguration.setString(key, value);
+        } catch (Throwable t) {
+            throw new SqlExecutionException("Could not set session property.", 
t);
+        }
+
+        // Renew the ExecutionContext by new environment.
+        // Book keep all the session states of current ExecutionContext then
+        // re-register them into the new one.
+        this.executionContext = new ExecutionContext(sessionEnv, 
executionContext);
+    }
+
+    /** Close resources, e.g. catalogs. */
+    public void close() {
+        try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(classLoader)) {
+            for (String name : sessionState.catalogManager.listCatalogs()) {
+                
sessionState.catalogManager.getCatalog(name).ifPresent(Catalog::close);
+            }
+        }
+        try {
+            classLoader.close();
+        } catch (IOException e) {
+            LOG.debug("Error while closing class loader.", e);
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Helper method to create
+    // 
--------------------------------------------------------------------------------------------
+
+    public static SessionContext create(DefaultContext defaultContext, String 
sessionId) {
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init config
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        Environment sessionEnv = defaultContext.getDefaultEnv().clone();
+        Configuration configuration = defaultContext.getFlinkConfig().clone();
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init classloader
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        URLClassLoader classLoader =
+                ClientUtils.buildUserCodeClassLoader(
+                        defaultContext.getDependencies(),
+                        Collections.emptyList(),
+                        SessionContext.class.getClassLoader(),
+                        configuration);
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init session state
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        ModuleManager moduleManager = new ModuleManager();
+
+        final EnvironmentSettings settings = 
sessionEnv.getExecution().getEnvironmentSettings();
+
+        CatalogManager catalogManager =
+                CatalogManager.newBuilder()
+                        .classLoader(classLoader)
+                        .config(configuration)
+                        .defaultCatalog(
+                                settings.getBuiltInCatalogName(),
+                                new GenericInMemoryCatalog(
+                                        settings.getBuiltInCatalogName(),
+                                        settings.getBuiltInDatabaseName()))
+                        .build();
+
+        FunctionCatalog functionCatalog =
+                new FunctionCatalog(configuration, catalogManager, 
moduleManager);
+        SessionState sessionState =
+                new SessionState(catalogManager, moduleManager, 
functionCatalog);
+
+        // 
--------------------------------------------------------------------------------------------------------------
+        // Init
+        // 
--------------------------------------------------------------------------------------------------------------
+
+        return new SessionContext(
+                defaultContext,
+                sessionId,
+                sessionEnv,
+                configuration,
+                classLoader,
+                sessionState,
+                new ExecutionContext(sessionEnv, configuration, classLoader, 
sessionState));
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof SessionContext)) {
+            return false;
+        }
+        SessionContext context = (SessionContext) o;
+        return Objects.equals(sessionId, context.sessionId)
+                && Objects.equals(defaultContext, context.defaultContext)
+                && Objects.equals(sessionEnv, context.sessionEnv)
+                && Objects.equals(sessionConfiguration, 
context.sessionConfiguration)
+                && Objects.equals(classLoader, context.classLoader)

Review comment:
       Do we need `hashCode` and `equals` for `SessionContext`? I'm afraid it 
doesn't work to compare classloader directly. 




----------------------------------------------------------------
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