lsyldliu commented on code in PR #19894:
URL: https://github.com/apache/flink/pull/19894#discussion_r925587246


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;

Review Comment:
   Adjust it after `dynamicConfig `?



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);
+
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {
+        DefaultContext context =
+                
DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig));
+        sessionManager = new SessionManager(context);
+
+        sessionManager.start();
+        SqlGatewayService sqlGatewayService = new 
SqlGatewayServiceImpl(sessionManager);
+
+        Configuration conf = context.getFlinkConfig();
+        endpoints.addAll(
+                
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(sqlGatewayService, 
conf));
+
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            try {
+                endpoint.start();
+            } catch (Throwable t) {
+                LOG.error("Failed to start the endpoint.", t);
+                stop();
+                throw new SqlGatewayException("Failed to start the endpoint.", 
t);
+            }
+        }
+    }
+
+    public void stop() {
+        latch.countDown();
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            stopEndpointSilently(endpoint);
+        }
+        if (sessionManager != null) {
+            sessionManager.stop();
+        }
+    }
+
+    public static void main(String[] args) {
+        startSqlGateway(System.out, args);
+    }
+
+    @VisibleForTesting
+    public static void startSqlGateway(PrintStream stream, String[] args) {

Review Comment:
   ```suggestion
      static void startSqlGateway(PrintStream stream, String[] args) {
   ```



##########
flink-table/flink-sql-gateway-api/src/test/java/org/apache/flink/table/gateway/api/utils/MockedSqlGatewayEndpoint.java:
##########
@@ -20,29 +20,45 @@
 
 import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Objects;
 
 /** Mocked {@link SqlGatewayEndpoint}. */
 public class MockedSqlGatewayEndpoint implements SqlGatewayEndpoint {
 
+    private static final Map<String, Boolean> RUNNING_ENDPOINTS = new 
HashMap<>();

Review Comment:
   Why not use `Set` directly? add the id to this set when start,  remove the 
id from this set when stop.



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);
+
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {
+        DefaultContext context =
+                
DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig));
+        sessionManager = new SessionManager(context);
+
+        sessionManager.start();
+        SqlGatewayService sqlGatewayService = new 
SqlGatewayServiceImpl(sessionManager);
+
+        Configuration conf = context.getFlinkConfig();
+        endpoints.addAll(
+                
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(sqlGatewayService, 
conf));
+
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            try {
+                endpoint.start();
+            } catch (Throwable t) {
+                LOG.error("Failed to start the endpoint.", t);
+                stop();
+                throw new SqlGatewayException("Failed to start the endpoint.", 
t);
+            }
+        }
+    }
+
+    public void stop() {
+        latch.countDown();
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            stopEndpointSilently(endpoint);
+        }
+        if (sessionManager != null) {
+            sessionManager.stop();
+        }
+    }
+
+    public static void main(String[] args) {
+        startSqlGateway(System.out, args);
+    }
+
+    @VisibleForTesting
+    public static void startSqlGateway(PrintStream stream, String[] args) {
+        SqlGatewayOptions cliOptions = 
SqlGatewayOptionsParser.parseSqlGatewayOptions(args);
+
+        if (cliOptions.isPrintHelp()) {
+            SqlGatewayOptionsParser.printHelpSqlGateway(stream);
+            return;
+        }
+
+        SqlGateway gateway = new SqlGateway(cliOptions.getDynamicConfigs());
+        try {
+            Runtime.getRuntime().addShutdownHook(new ShutdownThread(gateway));
+            gateway.start();
+            gateway.waitUntilStop();
+        } catch (Throwable t) {
+            // make space in terminal
+            stream.println();
+            stream.println();
+            LOG.error(
+                    "SqlGateway must stop. Unexpected exception. This is a 
bug. Please consider filing an issue.",
+                    t);
+            throw new SqlGatewayException(
+                    "Unexpected exception. This is a bug. Please consider 
filing an issue.", t);
+        }
+    }
+
+    private void waitUntilStop() throws Exception {
+        latch.await();
+    }
+
+    private void stopEndpointSilently(SqlGatewayEndpoint endpoint) {
+        try {
+            endpoint.stop();
+        } catch (Exception e) {
+            LOG.error("Failed to stop the endpoint. Ignore.", e);
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+
+    private static class ShutdownThread extends Thread {
+
+        private final SqlGateway gateway;
+
+        public ShutdownThread(SqlGateway gateway) {
+            this.gateway = gateway;
+        }
+
+        @Override
+        public void run() {
+            // Shutdown the gateway
+            System.out.println("\nShutting down the Flink SqlGateway...");
+            LOG.info("Shutting down the Flink SqlGateway...");
+
+            try {
+                gateway.stop();
+            } catch (Exception e) {
+                LOG.error("Failed to shut down the Flink SqlGateway: " + 
e.getMessage(), e);
+                System.out.println("Failed to shut down the Flink SqlGateway: 
" + e.getMessage());
+            }
+
+            LOG.info("Flink SqlServer has been shutdown.");

Review Comment:
   ```suggestion
               LOG.info("Flink SqlGateway has been shutdown.");
   ```



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);

Review Comment:
   ```suggestion
           this.latch = new CountDownLatch(1);
   ```



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);
+
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {
+        DefaultContext context =
+                
DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig));
+        sessionManager = new SessionManager(context);
+
+        sessionManager.start();
+        SqlGatewayService sqlGatewayService = new 
SqlGatewayServiceImpl(sessionManager);
+
+        Configuration conf = context.getFlinkConfig();
+        endpoints.addAll(
+                
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(sqlGatewayService, 
conf));
+
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            try {
+                endpoint.start();
+            } catch (Throwable t) {
+                LOG.error("Failed to start the endpoint.", t);
+                stop();
+                throw new SqlGatewayException("Failed to start the endpoint.", 
t);
+            }
+        }
+    }
+
+    public void stop() {
+        latch.countDown();
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            stopEndpointSilently(endpoint);
+        }
+        if (sessionManager != null) {
+            sessionManager.stop();
+        }
+    }
+
+    public static void main(String[] args) {
+        startSqlGateway(System.out, args);
+    }
+
+    @VisibleForTesting
+    public static void startSqlGateway(PrintStream stream, String[] args) {
+        SqlGatewayOptions cliOptions = 
SqlGatewayOptionsParser.parseSqlGatewayOptions(args);
+
+        if (cliOptions.isPrintHelp()) {
+            SqlGatewayOptionsParser.printHelpSqlGateway(stream);
+            return;
+        }
+
+        SqlGateway gateway = new SqlGateway(cliOptions.getDynamicConfigs());
+        try {
+            Runtime.getRuntime().addShutdownHook(new ShutdownThread(gateway));
+            gateway.start();
+            gateway.waitUntilStop();
+        } catch (Throwable t) {
+            // make space in terminal
+            stream.println();
+            stream.println();
+            LOG.error(
+                    "SqlGateway must stop. Unexpected exception. This is a 
bug. Please consider filing an issue.",
+                    t);
+            throw new SqlGatewayException(
+                    "Unexpected exception. This is a bug. Please consider 
filing an issue.", t);
+        }
+    }
+
+    private void waitUntilStop() throws Exception {
+        latch.await();
+    }
+
+    private void stopEndpointSilently(SqlGatewayEndpoint endpoint) {
+        try {
+            endpoint.stop();
+        } catch (Exception e) {
+            LOG.error("Failed to stop the endpoint. Ignore.", e);
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+
+    private static class ShutdownThread extends Thread {
+
+        private final SqlGateway gateway;
+
+        public ShutdownThread(SqlGateway gateway) {
+            this.gateway = gateway;
+        }
+
+        @Override
+        public void run() {
+            // Shutdown the gateway
+            System.out.println("\nShutting down the Flink SqlGateway...");
+            LOG.info("Shutting down the Flink SqlGateway...");
+
+            try {
+                gateway.stop();
+            } catch (Exception e) {
+                LOG.error("Failed to shut down the Flink SqlGateway: " + 
e.getMessage(), e);
+                System.out.println("Failed to shut down the Flink SqlGateway: 
" + e.getMessage());
+            }
+
+            LOG.info("Flink SqlServer has been shutdown.");
+            System.out.println("Flink SqlServer has been shutdown.");

Review Comment:
   ```suggestion
               System.out.println("Flink SqlGateway has been shutdown.");
   ```



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();

Review Comment:
   ```suggestion
           this.endpoints = new ArrayList<>();
   ```



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);
+
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {

Review Comment:
   Introducing a awaitTerminal api of Endpoint, when anyone of endpoint exit, 
the gateway also should exit.



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.gateway;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
+import 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils;
+import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptions;
+import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser;
+import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.session.SessionManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SqlGateway.class);
+
+    private final CountDownLatch latch;
+
+    private SessionManager sessionManager;
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    public SqlGateway(Properties dynamicConfig) {
+        endpoints = new ArrayList<>();
+        latch = new CountDownLatch(1);
+
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {
+        DefaultContext context =
+                
DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig));
+        sessionManager = new SessionManager(context);
+
+        sessionManager.start();
+        SqlGatewayService sqlGatewayService = new 
SqlGatewayServiceImpl(sessionManager);
+
+        Configuration conf = context.getFlinkConfig();
+        endpoints.addAll(
+                
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(sqlGatewayService, 
conf));

Review Comment:
   ```suggestion
                   
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(sqlGatewayService, 
context.getFlinkConfig()));
   ```



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to