Skip to content

Commit

Permalink
[FLINK-36760][sql-client] Supports to deploy script via sql client (a…
Browse files Browse the repository at this point in the history
  • Loading branch information
fsk119 authored and xaniasd committed Jan 13, 2025
1 parent d7fbfda commit a9c0450
Show file tree
Hide file tree
Showing 27 changed files with 648 additions and 181 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,9 @@
public enum PackagedProgramUtils {
;

private static final String SQL_DRIVER_CLASS_NAME =
"org.apache.flink.table.runtime.application.SqlDriver";

private static final String PYTHON_GATEWAY_CLASS_NAME =
"org.apache.flink.client.python.PythonGatewayServer";

Expand Down Expand Up @@ -193,43 +196,21 @@ public static boolean isPython(String[] programArguments) {
}

public static URL getPythonJar() {
String flinkOptPath = System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR);
final List<Path> pythonJarPath = new ArrayList<>();
try {
Files.walkFileTree(
FileSystems.getDefault().getPath(flinkOptPath),
new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
throws IOException {
FileVisitResult result = super.visitFile(file, attrs);
if (file.getFileName().toString().startsWith("flink-python")) {
pythonJarPath.add(file);
}
return result;
}
});
} catch (IOException e) {
throw new RuntimeException(
"Exception encountered during finding the flink-python jar. This should not happen.",
e);
}

if (pythonJarPath.size() != 1) {
throw new RuntimeException("Found " + pythonJarPath.size() + " flink-python jar.");
}

try {
return pythonJarPath.get(0).toUri().toURL();
} catch (MalformedURLException e) {
throw new RuntimeException("URL is invalid. This should not happen.", e);
}
return getOptJar("flink-python");
}

public static String getPythonDriverClassName() {
return PYTHON_DRIVER_CLASS_NAME;
}

public static boolean isSqlApplication(String entryPointClassName) {
return (entryPointClassName != null) && (entryPointClassName.equals(SQL_DRIVER_CLASS_NAME));
}

public static URL getSqlGatewayJar() {
return getOptJar("flink-sql-gateway");
}

public static URI resolveURI(String path) throws URISyntaxException {
final URI uri = new URI(path);
if (uri.getScheme() != null) {
Expand Down Expand Up @@ -260,4 +241,39 @@ private static ProgramInvocationException generateException(
stderr.length() == 0 ? "(none)" : stderr),
cause);
}

private static URL getOptJar(String jarName) {
String flinkOptPath = System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR);
final List<Path> optJarPath = new ArrayList<>();
try {
Files.walkFileTree(
FileSystems.getDefault().getPath(flinkOptPath),
new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
throws IOException {
FileVisitResult result = super.visitFile(file, attrs);
if (file.getFileName().toString().startsWith(jarName)) {
optJarPath.add(file);
}
return result;
}
});
} catch (IOException e) {
throw new RuntimeException(
"Exception encountered during finding the flink-python jar. This should not happen.",
e);
}

if (optJarPath.size() != 1) {
throw new RuntimeException(
String.format("Found " + optJarPath.size() + " %s jar.", jarName));
}

try {
return optJarPath.get(0).toUri().toURL();
} catch (MalformedURLException e) {
throw new RuntimeException("URL is invalid. This should not happen.", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@

import org.junit.Test;

import java.net.URI;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
Expand Down Expand Up @@ -69,11 +70,13 @@ protected List<String> formatRawResult(List<String> rawResult) {
}

@Override
protected void executeSqlStatements(ClusterController clusterController, List<String> sqlLines)
protected void executeSqlStatements(
ClusterController clusterController, List<String> sqlLines, List<URI> dependencies)
throws Exception {
clusterController.submitSQLJob(
new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
.addJar(SQL_TOOL_BOX_JAR)
.addJars(dependencies.toArray(new URI[0]))
.build(),
Duration.ofMinutes(2L));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.nio.file.Path;
import java.time.Duration;
import java.util.List;
Expand Down Expand Up @@ -91,10 +92,12 @@ protected void destroyHDFS() {
}

@Override
protected void executeSqlStatements(ClusterController clusterController, List<String> sqlLines)
protected void executeSqlStatements(
ClusterController clusterController, List<String> sqlLines, List<URI> dependencies)
throws Exception {
clusterController.submitSQLJob(
new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
.addJars(dependencies.toArray(new URI[0]))
.setEnvProcessor(
map -> map.put("HADOOP_CLASSPATH", getHadoopClassPathContent()))
.build(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@

import org.junit.Test;

import java.net.URI;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
Expand Down Expand Up @@ -76,11 +77,13 @@ protected List<String> formatRawResult(List<String> rawResult) {
}

@Override
protected void executeSqlStatements(ClusterController clusterController, List<String> sqlLines)
protected void executeSqlStatements(
ClusterController clusterController, List<String> sqlLines, List<URI> dependencies)
throws Exception {
clusterController.submitSQLJob(
new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
.addJar(SQL_TOOL_BOX_JAR)
.addJars(dependencies.toArray(new URI[0]))
.build(),
Duration.ofMinutes(2L));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.net.URL;
import java.nio.file.Files;
import java.nio.file.Path;
Expand Down Expand Up @@ -125,23 +126,25 @@ public void runAndCheckSQL(
runAndCheckSQL(
sqlPath,
Collections.singletonMap(result, resultItems),
Collections.singletonMap(result, formatter));
Collections.singletonMap(result, formatter),
Collections.emptyList());
}

public void runAndCheckSQL(String sqlPath, Map<Path, List<String>> resultItems)
throws Exception {
runAndCheckSQL(sqlPath, resultItems, Collections.emptyMap());
runAndCheckSQL(sqlPath, resultItems, Collections.emptyMap(), Collections.emptyList());
}

public void runAndCheckSQL(
String sqlPath,
Map<Path, List<String>> resultItems,
Map<Path, Function<List<String>, List<String>>> formatters)
Map<Path, Function<List<String>, List<String>>> formatters,
List<URI> dependencies)
throws Exception {
try (ClusterController clusterController = flink.startCluster(1)) {
List<String> sqlLines = initializeSqlLines(sqlPath);

executeSqlStatements(clusterController, sqlLines);
executeSqlStatements(clusterController, sqlLines, dependencies);

// Wait until all the results flushed to the json file.
LOG.info("Verify the result.");
Expand All @@ -163,7 +166,8 @@ protected Map<String, String> generateReplaceVars() {
}

protected abstract void executeSqlStatements(
ClusterController clusterController, List<String> sqlLines) throws Exception;
ClusterController clusterController, List<String> sqlLines, List<URI> dependencies)
throws Exception;

private List<String> initializeSqlLines(String sqlPath) throws IOException {
URL url = SqlITCaseBase.class.getClassLoader().getResource(sqlPath);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.junit.rules.TestName;

import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
Expand Down Expand Up @@ -90,6 +91,25 @@ public void testUdfInRemoteJar() throws Exception {
raw, USER_ORDER_SCHEMA, USER_ORDER_DESERIALIZATION_SCHEMA));
}

@Test
public void testCreateFunctionFromRemoteJarViaSqlClient() throws Exception {
runAndCheckSQL(
"sql_client_remote_jar_e2e.sql",
Collections.singletonMap(result, Arrays.asList("+I[Bob, 2]", "+I[Alice, 1]")),
Collections.singletonMap(
result,
raw ->
convertToMaterializedResult(
raw, USER_ORDER_SCHEMA, USER_ORDER_DESERIALIZATION_SCHEMA)),
Collections.singletonList(
URI.create(
String.format(
"hdfs://%s:%s/%s",
hdfsCluster.getURI().getHost(),
hdfsCluster.getNameNodePort(),
hdPath))));
}

@Test
public void testScalarUdfWhenCheckpointEnable() throws Exception {
runAndCheckSQL(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.
*/

CREATE TABLE JsonTable (
user_name STRING,
order_cnt BIGINT
) WITH (
'connector' = 'filesystem',
'path' = '$RESULT',
'sink.rolling-policy.rollover-interval' = '2s',
'sink.rolling-policy.check-interval' = '2s',
'format' = 'debezium-json'
);

create function count_agg as 'org.apache.flink.table.toolbox.CountAggFunction' LANGUAGE JAVA;

SET execution.runtime-mode = $MODE;
SET table.exec.mini-batch.enabled = true;
SET table.exec.mini-batch.size = 5;
SET table.exec.mini-batch.allow-latency = 2s;

INSERT INTO JsonTable
SELECT user_name, count_agg(order_id)
FROM (VALUES (1, 'Bob'), (2, 'Bob'), (1, 'Alice')) T(order_id, user_name)
GROUP BY user_name;
Original file line number Diff line number Diff line change
Expand Up @@ -26,14 +26,12 @@
import org.apache.flink.table.client.gateway.DefaultContextUtils;
import org.apache.flink.table.client.gateway.Executor;
import org.apache.flink.table.client.gateway.SingleSessionManager;
import org.apache.flink.table.client.gateway.SqlExecutionException;
import org.apache.flink.table.gateway.SqlGateway;
import org.apache.flink.table.gateway.rest.SqlGatewayRestEndpointFactory;
import org.apache.flink.table.gateway.rest.util.SqlGatewayRestOptions;
import org.apache.flink.table.gateway.service.context.DefaultContext;
import org.apache.flink.util.NetUtils;

import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.SystemUtils;
import org.jline.terminal.Terminal;
import org.slf4j.Logger;
Expand All @@ -42,16 +40,14 @@
import javax.annotation.Nullable;

import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URL;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.function.Supplier;

import static org.apache.flink.table.client.cli.CliClient.DEFAULT_TERMINAL_FACTORY;
import static org.apache.flink.table.client.cli.CliUtils.isApplicationMode;
import static org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils.getSqlGatewayOptionPrefix;

/**
Expand Down Expand Up @@ -140,7 +136,11 @@ private void openCli(Executor executor) {

try (CliClient cli = new CliClient(terminalFactory, executor, historyFilePath)) {
if (options.getInitFile() != null) {
boolean success = cli.executeInitialization(readFromURL(options.getInitFile()));
if (isApplicationMode(executor.getSessionConfig())) {
throw new SqlClientException(
"Sql Client doesn't support to run init files when deploying script into cluster.");
}
boolean success = cli.executeInitialization(options.getInitFile());
if (!success) {
System.out.println(
String.format(
Expand All @@ -158,7 +158,7 @@ private void openCli(Executor executor) {
if (!hasSqlFile) {
cli.executeInInteractiveMode();
} else {
cli.executeInNonInteractiveMode(readExecutionContent());
cli.executeInNonInteractiveMode(options.getSqlFile());
}
}
}
Expand Down Expand Up @@ -320,17 +320,4 @@ public void run() {
System.out.println("done.");
}
}

private String readExecutionContent() {
return readFromURL(options.getSqlFile());
}

private String readFromURL(URL file) {
try {
return IOUtils.toString(file, StandardCharsets.UTF_8);
} catch (IOException e) {
throw new SqlExecutionException(
String.format("Fail to read content from the %s.", file.getPath()), e);
}
}
}
Loading

0 comments on commit a9c0450

Please sign in to comment.