hackergin commented on code in PR #25730:
URL: https://github.com/apache/flink/pull/25730#discussion_r1885000844
##########
flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/SqlGatewayService.java:
##########
@@ -347,4 +348,23 @@ OperationHandle refreshMaterializedTable(
Map<String, String> dynamicOptions,
Map<String, String> staticPartitions,
Map<String, String> executionConfig);
+
+ //
-------------------------------------------------------------------------------------------
+ // Deploy Script
+ //
-------------------------------------------------------------------------------------------
+
+ /**
+ * Deploy the script in application mode.
+ *
+ * @param sessionHandle handle to identify the session.
+ * @param scriptPath path to the script.
+ * @param executionConfig to run the script.
Review Comment:
It seems that the documentation is missing a description for the script
parameter.
##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/DeployScriptITCase.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.rest;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.client.deployment.ClusterClientFactory;
+import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.execution.CheckpointType;
+import org.apache.flink.core.execution.SavepointFormatType;
+import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.streaming.api.graph.ExecutionPlan;
+import org.apache.flink.table.gateway.api.session.SessionHandle;
+import
org.apache.flink.table.gateway.rest.header.application.DeployScriptHeaders;
+import org.apache.flink.table.gateway.rest.header.session.OpenSessionHeaders;
+import
org.apache.flink.table.gateway.rest.message.application.DeployScriptRequestBody;
+import
org.apache.flink.table.gateway.rest.message.session.OpenSessionRequestBody;
+import
org.apache.flink.table.gateway.rest.message.session.SessionMessageParameters;
+import
org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension;
+import org.apache.flink.table.gateway.rest.util.TestingRestClient;
+import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension;
+import org.apache.flink.table.runtime.application.SqlDriver;
+
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Order;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase to test deploy the script into application mode. */
+public class DeployScriptITCase {
+
+ @RegisterExtension
+ @Order(1)
+ public static final SqlGatewayServiceExtension
SQL_GATEWAY_SERVICE_EXTENSION =
+ new SqlGatewayServiceExtension(Configuration::new);
+
+ @RegisterExtension
+ @Order(2)
+ private static final SqlGatewayRestEndpointExtension
SQL_GATEWAY_REST_ENDPOINT_EXTENSION =
+ new
SqlGatewayRestEndpointExtension(SQL_GATEWAY_SERVICE_EXTENSION::getService);
+
+ private static TestingRestClient restClient;
+ private static SessionHandle sessionHandle;
+
+ @BeforeAll
+ static void beforeAll() throws Exception {
+ restClient = TestingRestClient.getTestingRestClient();
+ sessionHandle =
+ new SessionHandle(
+ UUID.fromString(
+ restClient
+ .sendRequest(
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION
+ .getTargetAddress(),
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetPort(),
+
OpenSessionHeaders.getInstance(),
+
EmptyMessageParameters.getInstance(),
+ new OpenSessionRequestBody(
+ "test",
+
Collections.singletonMap("key", "value")))
+ .get()
+ .getSessionHandle()));
+ }
+
+ @Test
+ void testDeployScriptToYarnCluster() throws Exception {
+ verifyDeployScriptToCluster("yarn-application");
+ }
+
+ @Test
+ void testDeployScriptToKubernetesCluster() throws Exception {
+ verifyDeployScriptToCluster("kubernetes-application");
+ }
+
+ private void verifyDeployScriptToCluster(String target) throws Exception {
+ TestApplicationClusterClientFactory.id = target;
+ String script =
+ "CREATE TEMPORARY TABLE sink(\n"
+ + " a INT\n"
+ + ") WITH (\n"
+ + " 'connector' = 'blackhole'\n"
+ + ")\n"
Review Comment:
nit: It seems that a `;` is missing.
##########
flink-table/flink-sql-gateway/src/test/resources/sql_gateway_rest_api_v4.snapshot:
##########
@@ -341,6 +341,46 @@
}
}
}
+ }, {
+ "url" : "/sessions/:session_handle/scripts",
+ "method" : "POST",
+ "status-code" : "202 Accepted",
Review Comment:
I noticed that the return code in the FLIP documentation is 200, but here it
is 202. Could you confirm if this is expected?
##########
flink-table/flink-sql-gateway-api/src/test/java/org/apache/flink/table/gateway/api/utils/MockedSqlGatewayService.java:
##########
@@ -201,6 +202,16 @@ public OperationHandle refreshMaterializedTable(
throw new UnsupportedOperationException();
}
+ @Override
+ public <ClusterID> ClusterID deployScript(
+ SessionHandle sessionHandle,
+ @org.jetbrains.annotations.Nullable Path scriptPath,
Review Comment:
Consider using javax.annotation.Nullable instead of
org.jetbrains.annotations.Nullable to maintain consistency with other parts of
the code
##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/DeployScriptITCase.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.rest;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.client.deployment.ClusterClientFactory;
+import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.execution.CheckpointType;
+import org.apache.flink.core.execution.SavepointFormatType;
+import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.streaming.api.graph.ExecutionPlan;
+import org.apache.flink.table.gateway.api.session.SessionHandle;
+import
org.apache.flink.table.gateway.rest.header.application.DeployScriptHeaders;
+import org.apache.flink.table.gateway.rest.header.session.OpenSessionHeaders;
+import
org.apache.flink.table.gateway.rest.message.application.DeployScriptRequestBody;
+import
org.apache.flink.table.gateway.rest.message.session.OpenSessionRequestBody;
+import
org.apache.flink.table.gateway.rest.message.session.SessionMessageParameters;
+import
org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension;
+import org.apache.flink.table.gateway.rest.util.TestingRestClient;
+import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension;
+import org.apache.flink.table.runtime.application.SqlDriver;
+
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Order;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase to test deploy the script into application mode. */
+public class DeployScriptITCase {
+
+ @RegisterExtension
+ @Order(1)
+ public static final SqlGatewayServiceExtension
SQL_GATEWAY_SERVICE_EXTENSION =
+ new SqlGatewayServiceExtension(Configuration::new);
+
+ @RegisterExtension
+ @Order(2)
+ private static final SqlGatewayRestEndpointExtension
SQL_GATEWAY_REST_ENDPOINT_EXTENSION =
+ new
SqlGatewayRestEndpointExtension(SQL_GATEWAY_SERVICE_EXTENSION::getService);
+
+ private static TestingRestClient restClient;
+ private static SessionHandle sessionHandle;
+
+ @BeforeAll
+ static void beforeAll() throws Exception {
+ restClient = TestingRestClient.getTestingRestClient();
+ sessionHandle =
+ new SessionHandle(
+ UUID.fromString(
+ restClient
+ .sendRequest(
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION
+ .getTargetAddress(),
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetPort(),
+
OpenSessionHeaders.getInstance(),
+
EmptyMessageParameters.getInstance(),
+ new OpenSessionRequestBody(
+ "test",
+
Collections.singletonMap("key", "value")))
+ .get()
+ .getSessionHandle()));
+ }
+
+ @Test
+ void testDeployScriptToYarnCluster() throws Exception {
+ verifyDeployScriptToCluster("yarn-application");
+ }
+
+ @Test
+ void testDeployScriptToKubernetesCluster() throws Exception {
+ verifyDeployScriptToCluster("kubernetes-application");
+ }
+
+ private void verifyDeployScriptToCluster(String target) throws Exception {
+ TestApplicationClusterClientFactory.id = target;
+ String script =
+ "CREATE TEMPORARY TABLE sink(\n"
+ + " a INT\n"
+ + ") WITH (\n"
+ + " 'connector' = 'blackhole'\n"
+ + ")\n"
+ + "INSERT INTO sink VALUES (1);";
+ assertThat(
+ restClient
+ .sendRequest(
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetAddress(),
+
SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetPort(),
+ DeployScriptHeaders.getInstance(),
+ new
SessionMessageParameters(sessionHandle),
+ new DeployScriptRequestBody(
+ script,
+ null,
Review Comment:
It’s recommended to pass a test ExecutionConfig here instead of null to
verify whether executionConfig takes effect correctly.
##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/SqlGatewayServiceImpl.java:
##########
@@ -330,6 +336,41 @@ public OperationHandle refreshMaterializedTable(
}
}
+ @Override
+ public <ClusterID> ClusterID deployScript(
+ SessionHandle sessionHandle,
+ @Nullable Path scriptPath,
+ @Nullable String script,
+ Configuration executionConfig)
+ throws SqlGatewayException {
+ Session session = sessionManager.getSession(sessionHandle);
+ if (scriptPath == null && script == null) {
+ throw new IllegalArgumentException("Please specify script path or
script.");
+ }
+ Configuration mergedConfig =
Configuration.fromMap(session.getSessionConfig());
+ mergedConfig.addAll(executionConfig);
+
+ List<String> arguments = new ArrayList<>();
+ if (scriptPath != null) {
+ arguments.add("--" + SqlDriver.OPTION_SQL_FILE.getLongOpt());
+ arguments.add(scriptPath.toString());
+ }
+ if (script != null) {
Review Comment:
Is it necessary to restrict that only one of scriptPath or script can be set?
--
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]