Skip to content

Commit

Permalink
[FLINK-36760][sql-client] Supports to deploy script via sql client
Browse files Browse the repository at this point in the history
  • Loading branch information
fsk119 committed Dec 6, 2024
1 parent fa4d8ea commit 0bea07d
Show file tree
Hide file tree
Showing 22 changed files with 394 additions and 134 deletions.
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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,19 @@
package org.apache.flink.table.client.cli;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.table.client.SqlClientException;
import org.apache.flink.table.client.cli.parser.SqlClientSyntaxHighlighter;
import org.apache.flink.table.client.cli.parser.SqlCommandParserImpl;
import org.apache.flink.table.client.cli.parser.SqlMultiLineParser;
import org.apache.flink.table.client.config.SqlClientOptions;
import org.apache.flink.table.client.gateway.Executor;
import org.apache.flink.table.client.gateway.SqlExecutionException;
import org.apache.flink.util.FileUtils;

import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.jline.reader.EndOfFileException;
import org.jline.reader.LineReader;
import org.jline.reader.LineReaderBuilder;
Expand All @@ -47,10 +52,17 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.HttpURLConnection;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.function.Supplier;

import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_DEPLOY_SCRIPT;
import static org.apache.flink.table.client.cli.CliStrings.messageInfo;
import static org.apache.flink.table.client.cli.CliUtils.isApplicationMode;

/** SQL CLI client. */
public class CliClient implements AutoCloseable {

Expand Down Expand Up @@ -125,21 +137,36 @@ void executeInInteractiveMode(LineReader lineReader) {
}

/** Opens the non-interactive CLI shell. */
public void executeInNonInteractiveMode(String content) {
public void executeInNonInteractiveMode(URI uri) {
try {
terminal = terminalFactory.get();
executeFile(content, terminal.output(), ExecutionMode.NON_INTERACTIVE_EXECUTION);
if (isApplicationMode(executor.getSessionConfig())) {
String scheme = StringUtils.lowerCase(uri.getScheme());
String clusterId;
// local files
if (scheme.equals("file")) {
clusterId = executor.deployScript(readFile(uri), null);
} else {
clusterId = executor.deployScript(null, uri);
}
terminal.writer().println(messageInfo(MESSAGE_DEPLOY_SCRIPT + clusterId).toAnsi());
terminal.flush();
} else {
executeFile(
readFile(uri), terminal.output(), ExecutionMode.NON_INTERACTIVE_EXECUTION);
}
} finally {
closeTerminal();
}
}

/** Initialize the Cli Client with the content. */
public boolean executeInitialization(String content) {
public boolean executeInitialization(URI file) {
try {
OutputStream outputStream = new ByteArrayOutputStream(256);
terminal = TerminalUtils.createDumbTerminal(outputStream);
boolean success = executeFile(content, outputStream, ExecutionMode.INITIALIZATION);
boolean success =
executeFile(readFile(file), outputStream, ExecutionMode.INITIALIZATION);
LOG.info(outputStream.toString());
return success;
} finally {
Expand Down Expand Up @@ -326,4 +353,39 @@ private LineReader createLineReader(Terminal terminal, ExecutionMode mode) {
}
return lineReader;
}

public static String readFile(URI uri) {
try {
if (uri.getScheme() != null
&& (uri.getScheme().equals("http") || uri.getScheme().equals("https"))) {
return readFromHttp(uri);
} else {
return readFileUtf8(uri);
}
} catch (IOException e) {
throw new SqlClientException("Failed to read file " + uri, e);
}
}

private static String readFromHttp(URI uri) throws IOException {
HttpURLConnection conn = (HttpURLConnection) uri.toURL().openConnection();

conn.setRequestMethod("GET");

try (InputStream inputStream = conn.getInputStream();
ByteArrayOutputStream targetFile = new ByteArrayOutputStream()) {
IOUtils.copy(inputStream, targetFile);
return targetFile.toString(StandardCharsets.UTF_8);
}
}

private static String readFileUtf8(URI uri) throws IOException {
org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(uri.toString());
FileSystem fs = path.getFileSystem();
try (FSDataInputStream inputStream = fs.open(path)) {
return new String(
FileUtils.read(inputStream, (int) fs.getFileStatus(path).getLen()),
StandardCharsets.UTF_8);
}
}
}
Loading

0 comments on commit 0bea07d

Please sign in to comment.