You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/21 13:02:09 UTC

[GitHub] [flink] wuchong commented on a diff in pull request #19894: [FLINK-27931][sql-gateway] Introduce the SqlGateway to assemble all components

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


##########
flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/endpoint/SqlGatewayEndpoint.java:
##########
@@ -29,4 +29,7 @@ public interface SqlGatewayEndpoint {
 
     /** Terminate the endpoint. */
     void stop() throws Exception;
+
+    /** Waits for the server to become terminated. */
+    void awaitTermination() throws Exception;

Review Comment:
   I thought about this again. It's error-prone for developers to implement this. For example, the implementation of this method may not block before termination (by mistake) which will result in the SqlGateway exit unexpected. 
   
   We can implement a simple `awaitTermination` in `SqlGateway` via `lock.wait()` and `lock.notifyAll()`.
   



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.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;
+
+/** Main entry point for the SQL Gateway. */
+public class SqlGateway {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SqlGateway.class);
+
+    private final List<SqlGatewayEndpoint> endpoints;
+    private final Properties dynamicConfig;
+
+    private SessionManager sessionManager;
+
+    public SqlGateway(Properties dynamicConfig) {
+        this.endpoints = new ArrayList<>();
+        this.dynamicConfig = dynamicConfig;
+    }
+
+    public void start() {
+        DefaultContext context =
+                DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig));
+        sessionManager = new SessionManager(context);
+
+        sessionManager.start();
+        SqlGatewayService sqlGatewayService = new SqlGatewayServiceImpl(sessionManager);
+
+        endpoints.addAll(
+                SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint(
+                        sqlGatewayService, context.getFlinkConfig()));
+
+        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() {
+        for (SqlGatewayEndpoint endpoint : endpoints) {
+            stopEndpointSilently(endpoint);
+        }
+        if (sessionManager != null) {
+            sessionManager.stop();
+        }
+    }
+
+    public static void main(String[] args) {

Review Comment:
   I think we can follow `org.apache.flink.runtime.taskexecutor.TaskManagerRunner#main` to log some important environment information. 



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

Review Comment:
   The `HashSet` is not thread-safe but is modified and accessed by multiple threads. You can use `ConcurrentHashMap.newKeySet()` instead.



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/SqlGatewayTest.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.table.gateway.api.utils.MockedSqlGatewayEndpoint;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Tests for the {@link SqlGateway}. */
+public class SqlGatewayTest {
+
+    private Map<String, String> originalEnv;
+    private ByteArrayOutputStream output;
+
+    @BeforeEach
+    public void before(@TempDir File tempFolder) throws IOException {
+        originalEnv = System.getenv();
+
+        // prepare yaml
+        File confYaml = new File(tempFolder, "flink-conf.yaml");
+        if (!confYaml.createNewFile()) {
+            throw new IOException("Can't create testing flink-conf.yaml file.");
+        }
+        // adjust the test environment for the purposes of this test
+        Map<String, String> map = new HashMap<>(System.getenv());
+        map.put(ENV_FLINK_CONF_DIR, tempFolder.getAbsolutePath());
+        CommonTestUtils.setEnv(map);
+
+        output = new ByteArrayOutputStream(256);
+    }
+
+    @AfterEach
+    public void cleanup() throws Exception {
+        CommonTestUtils.setEnv(originalEnv);
+        if (output != null) {
+            output.close();
+        }
+    }
+
+    @Test
+    public void testPrintStartGatewayHelp() {
+        String[] args = new String[] {"--help"};
+        SqlGateway.startSqlGateway(new PrintStream(output), args);
+
+        assertThat(output.toString())
+                .contains(
+                        "Start the Flink SQL Gateway as a daemon to submit Flink SQL.\n"
+                                + "\n"
+                                + "  Syntax: start [OPTIONS]\n"
+                                + "     -D <property=value>   use value for given property\n"

Review Comment:
   ```suggestion
                                   + "     -D<property=value>   use value for given property\n"
   ```
   
   Shouldn't have space between `-D` and properties? 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org