godfreyhe commented on a change in pull request #15006: URL: https://github.com/apache/flink/pull/15006#discussion_r589133210
########## 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); + } Review comment: just re-initialize the `sessionConfiguration` based on `defaultContext` ? `sessionConfiguration = new Configuration(defaultContext.getFlinkConfig())` ########## File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java ########## @@ -409,20 +263,19 @@ public ResultDescriptor executeQuery(String sessionId, String query) @Override public void cancelQuery(String sessionId, String resultId) throws SqlExecutionException { - final ExecutionContext<?> context = getExecutionContext(sessionId); - cancelQueryInternal(context, resultId); + cancelQueryInternal(resultId); } @VisibleForTesting List<ModuleEntry> listFullModules(String sessionId) throws SqlExecutionException { - final ExecutionContext<?> context = getExecutionContext(sessionId); + final ExecutionContext context = getExecutionContext(sessionId); final TableEnvironment tableEnv = context.getTableEnvironment(); return context.wrapClassLoader(() -> Arrays.asList(tableEnv.listFullModules())); } // -------------------------------------------------------------------------------------------- - private <T> void cancelQueryInternal(ExecutionContext<T> context, String resultId) { + private <T> void cancelQueryInternal(String resultId) { Review comment: `<T>` should be removed ########## 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; Review comment: give some explanation why we need them both ########## File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java ########## @@ -121,6 +132,114 @@ private void start() { } } + private List<URL> discoverDependencies(List<URL> jars, List<URL> libraries) { Review comment: can we move these new methods into a utility class, this class has been too complex. ########## 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; + + public TableConfig() { + this.configuration = new Configuration(); Review comment: nit: call `this(new Configuration())` ########## 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 { + + // Members that should be reused in the same session. + private final Environment environment; + private final Configuration flinkConfig; + private final SessionState sessionState; + private final URLClassLoader classLoader; + + private final TableEnvironment tableEnv; + + public ExecutionContext( + Environment environment, + Configuration flinkConfig, + URLClassLoader classLoader, + SessionState sessionState) { + this.environment = environment; + this.flinkConfig = flinkConfig; + this.sessionState = sessionState; + this.classLoader = classLoader; + // Initialize the TableEnvironment. + this.tableEnv = createTableEnvironment(); + LegacyTableEnvironmentInitializer.initializeTableEnvironmentFromEnvironment( + tableEnv, environment, classLoader); + } + + /** Create a new {@link ExecutionContext} without initialization. */ + public ExecutionContext(Environment environment, ExecutionContext context) { Review comment: I would like to remove this method, because the above method is more generic and the `initializeTableEnvironmentFromEnvironment` method will be called in `SessionContext#create`. Actually, we can rename `initializeTableEnvironmentFromEnvironment` to `initializeSessionState`, because TableEnvironment instance will change with ExecutionContext ########## File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java ########## @@ -441,7 +294,7 @@ public void cancelQuery(String sessionId, String resultId) throws SqlExecutionEx } private <C> ResultDescriptor executeQueryInternal( Review comment: `<C>` should be removed ########## 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: will we introduce `TableEnvironmentInitializer` for ddl ? if that, it's better we could introduce an interface for LegacyTableEnvironmentInitializer and TableEnvironmentInitializer ---------------------------------------------------------------- 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