hackergin commented on code in PR #25724:
URL: https://github.com/apache/flink/pull/25724#discussion_r1882290443


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/application/ScriptRunner.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.service.application;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.gateway.api.session.SessionEnvironment;
+import org.apache.flink.table.gateway.api.session.SessionHandle;
+import org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion;
+import org.apache.flink.table.gateway.service.context.DefaultContext;
+import org.apache.flink.table.gateway.service.context.SessionContext;
+import org.apache.flink.util.concurrent.Executors;
+
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.UUID;
+
+/** Runner to run the script. It prepares the required dependencies and 
environment. */
+public class ScriptRunner {
+
+    private static final SessionHandle SESSION_HANDLE =
+            new 
SessionHandle(UUID.fromString("013059f8-760f-4390-b74d-d0818bd99365"));

Review Comment:
   Does fixing the UUID here have any special meaning?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/application/SqlDriver.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.runtime.application;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.MutableURLClassLoader;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.FileSystems;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Driver to load the {@code ScriptRunner} and execute the script. */
+public class SqlDriver {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SqlDriver.class);
+
+    public static final Option OPTION_SQL_FILE =
+            Option.builder()
+                    .longOpt("scriptPath")
+                    .numberOfArgs(1)
+                    .desc("SQL script file path.")
+                    .build();
+
+    public static final Option OPTION_SQL_STATEMENTS =
+            Option.builder().longOpt("script").numberOfArgs(1).desc("Script 
content.").build();
+
+    private static final String RUNNER_CLASS_NAME =
+            "org.apache.flink.table.gateway.service.application.ScriptRunner";
+    private static boolean testMode = false;
+    private static OutputStream testOutputStream;
+
+    public static void main(String[] args) throws Exception {
+        String sql = parseOptions(args);
+        if (testMode) {
+            getClassLoader()
+                    .loadClass(RUNNER_CLASS_NAME)
+                    .getMethod("run", String.class, OutputStream.class)
+                    .invoke(null, sql, 
Preconditions.checkNotNull(testOutputStream));
+        } else {
+            getClassLoader()
+                    .loadClass(RUNNER_CLASS_NAME)
+                    .getMethod("run", String.class)
+                    .invoke(null, sql);
+        }
+    }
+
+    public static void enableTestMode(OutputStream outputStream) {
+        testOutputStream = outputStream;
+        testMode = true;
+    }
+
+    public static void disableTestMode() {
+        testOutputStream = null;
+        SqlDriver.testMode = false;
+    }
+
+    private static ClassLoader getClassLoader() throws Exception {
+        MutableURLClassLoader sqlGatewayClassloader =
+                (MutableURLClassLoader) 
Thread.currentThread().getContextClassLoader();
+        try {
+            sqlGatewayClassloader.loadClass(RUNNER_CLASS_NAME);
+            LOG.info("Load {} from the classpath.", RUNNER_CLASS_NAME);
+        } catch (ClassNotFoundException e) {
+            LOG.info("{} is not in the classpath. Finding...", 
RUNNER_CLASS_NAME);
+            sqlGatewayClassloader.addURL(findExecutor().toUri().toURL());
+        }
+        return sqlGatewayClassloader;
+    }
+
+    private static Path findExecutor() {
+        String flinkOptPath = System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR);
+        final List<Path> sqlJarPaths = new ArrayList<>();
+        try {
+            Files.walkFileTree(
+                    FileSystems.getDefault().getPath(flinkOptPath),
+                    new SimpleFileVisitor<>() {
+                        @Override
+                        public FileVisitResult visitFile(Path file, 
BasicFileAttributes attrs)
+                                throws IOException {
+                            FileVisitResult result = super.visitFile(file, 
attrs);
+                            if 
(file.getFileName().toString().startsWith("flink-sql-gateway")) {
+                                sqlJarPaths.add(file);
+                            }
+                            return result;
+                        }
+                    });
+        } catch (IOException e) {
+            throw new RuntimeException(
+                    "Exception encountered during finding the 
flink-sql-gateway jar. This should not happen.",
+                    e);
+        }
+
+        if (sqlJarPaths.size() != 1) {
+            throw new RuntimeException("Found " + sqlJarPaths.size() + " 
flink-sql-gateway jar.");
+        }
+
+        return sqlJarPaths.get(0);
+    }
+
+    static Options getSqlDriverOptions() {
+        Options options = new Options();
+        options.addOption(OPTION_SQL_FILE);
+        options.addOption(OPTION_SQL_STATEMENTS);
+        return options;
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    //  Line Parsing
+    // 
--------------------------------------------------------------------------------------------
+
+    public static String parseOptions(String[] args) {
+        try {
+            DefaultParser parser = new DefaultParser();
+            CommandLine line = parser.parse(getSqlDriverOptions(), args, true);
+            String content = 
getContent(line.getOptionValue(OPTION_SQL_FILE.getLongOpt()));
+            if (content == null) {
+                return Preconditions.checkNotNull(
+                        
line.getOptionValue(OPTION_SQL_STATEMENTS.getLongOpt()),
+                        "Please set script or script path either.");

Review Comment:
   Modify the prompt to make it clearer, suggesting the use of --script or 
--scriptPath to set the parameters?



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/application/ScriptRunnerITCase.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.service.application;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.client.cli.ExecutionConfigAccessor;
+import 
org.apache.flink.client.deployment.application.ApplicationDispatcherLeaderProcessFactoryFactory;
+import 
org.apache.flink.client.deployment.application.executors.EmbeddedExecutor;
+import org.apache.flink.client.program.PackagedProgram;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.configuration.PipelineOptionsInternal;
+import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory;
+import 
org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory;
+import 
org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory;
+import 
org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory;
+import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.TestingMiniCluster;
+import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration;
+import 
org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory;
+import org.apache.flink.runtime.rest.JobRestEndpointFactory;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.table.gateway.service.utils.MockHttpServer;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.runtime.application.SqlDriver;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.UserClassLoaderJarTestUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+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.OutputStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Supplier;
+
+import static 
org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR;
+import static 
org.apache.flink.table.utils.UserDefinedFunctions.GENERATED_LOWER_UDF_CLASS;
+import static 
org.apache.flink.table.utils.UserDefinedFunctions.GENERATED_LOWER_UDF_CODE;
+import static 
org.apache.flink.table.utils.UserDefinedFunctions.GENERATED_UPPER_UDF_CLASS;
+import static 
org.apache.flink.table.utils.UserDefinedFunctions.GENERATED_UPPER_UDF_CODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase to verify {@link ScriptRunner}. */
+class ScriptRunnerITCase {
+
+    private static Map<String, String> originalEnv;
+    private static File udfJar;
+
+    private OutputStream outputStream;
+
+    @BeforeAll
+    static void beforeAll(@TempDir File flinkHome, @TempDir Path functionHome) 
throws Exception {
+        originalEnv = System.getenv();
+        // prepare yaml
+        File confYaml = new File(flinkHome, "config.yaml");
+        if (!confYaml.createNewFile()) {
+            throw new IOException("Can't create testing config.yaml file.");
+        }
+        Map<String, String> map = new HashMap<>(System.getenv());
+        map.put(ENV_FLINK_CONF_DIR, flinkHome.getAbsolutePath());
+        org.apache.flink.core.testutils.CommonTestUtils.setEnv(map);
+
+        Map<String, String> classNameCodes = new HashMap<>();
+        classNameCodes.put(
+                GENERATED_LOWER_UDF_CLASS,
+                String.format(GENERATED_LOWER_UDF_CODE, 
GENERATED_LOWER_UDF_CLASS));
+        classNameCodes.put(
+                GENERATED_UPPER_UDF_CLASS,
+                String.format(GENERATED_UPPER_UDF_CODE, 
GENERATED_UPPER_UDF_CLASS));
+
+        udfJar =
+                UserClassLoaderJarTestUtils.createJarFile(
+                        Files.createTempDirectory(functionHome, 
"test-jar").toFile(),
+                        "test-classloader-udf.jar",
+                        classNameCodes);
+    }
+
+    @BeforeEach
+    void beforeEach() {
+        outputStream = new ByteArrayOutputStream(1024);
+        SqlDriver.enableTestMode(outputStream);
+    }
+
+    @AfterEach
+    void afterEach() throws Exception {
+        outputStream.close();
+        SqlDriver.disableTestMode();
+    }
+
+    @AfterAll
+    static void afterAll() {
+        org.apache.flink.core.testutils.CommonTestUtils.setEnv(originalEnv);
+    }
+
+    @Test
+    void testRunScriptFromFile(@TempDir Path workDir) throws Exception {
+        String script =
+                String.format(
+                        "CREATE TEMPORARY TABLE sink(\n"
+                                + "  a STRING\n"
+                                + ") WITH (\n"
+                                + "  'connector' = 'values'\n"
+                                + ");\n"
+                                + "ADD JAR '%s';\n"
+                                + "CREATE TEMPORARY FUNCTION lower_func AS 
'%s';\n"
+                                + "CREATE TEMPORARY VIEW v(c) AS VALUES ('A'), 
('B'), ('C');\n"
+                                + "INSERT INTO sink SELECT lower_func(c) FROM 
v;",
+                        udfJar.getAbsolutePath(), GENERATED_LOWER_UDF_CLASS);
+
+        List<String> arguments =
+                Arrays.asList("--scriptPath", createStatementFile(workDir, 
script).toString());
+        runScriptInCluster(arguments);
+
+        assertThat(TestValuesTableFactory.getResultsAsStrings("sink"))
+                .containsExactly("+I[a]", "+I[b]", "+I[c]");
+    }
+
+    @Test
+    void testRunScriptFromRemoteFile(@TempDir Path workDir) throws Exception {
+        String script =
+                String.format(
+                        "CREATE TEMPORARY TABLE sink(\n"
+                                + "  a STRING\n"
+                                + ") WITH (\n"
+                                + "  'connector' = 'values'\n"
+                                + ");\n"
+                                + "ADD JAR '%s';\n"
+                                + "CREATE TEMPORARY FUNCTION lower_func AS 
'%s';\n"
+                                + "CREATE TEMPORARY VIEW v(c) AS VALUES ('A'), 
('B'), ('C');\n"
+                                + "INSERT INTO sink SELECT lower_func(c) FROM 
v;",
+                        udfJar.getAbsolutePath(), GENERATED_LOWER_UDF_CLASS);
+        File file = createStatementFile(workDir, script).toFile();
+
+        try (MockHttpServer server = MockHttpServer.startHttpServer()) {

Review Comment:
   That’s awesome! Using a mock HTTP server to test remote files is a great 
approach.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/application/SqlDriver.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.runtime.application;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.MutableURLClassLoader;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.FileSystems;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Driver to load the {@code ScriptRunner} and execute the script. */
+public class SqlDriver {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SqlDriver.class);
+
+    public static final Option OPTION_SQL_FILE =
+            Option.builder()
+                    .longOpt("scriptPath")
+                    .numberOfArgs(1)
+                    .desc("SQL script file path.")

Review Comment:
   The description here could be more detailed, such as mentioning support for 
DFS files and HTTP files.



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