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/19 07:00:42 UTC

[GitHub] [flink] fsk119 commented on a diff in pull request #20159: [FLINK-28360][SQL Client] Support stop job statement in SQL client

fsk119 commented on code in PR #20159:
URL: https://github.com/apache/flink/pull/20159#discussion_r924018891


##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/tcl/SqlStopJob.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.sql.parser.tcl;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.NlsString;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collections;
+import java.util.List;
+
+/** Stop job command to stop a flink job. */

Review Comment:
   How about:
   
   ```
   The command to stop a flink job.
   ```



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java:
##########
@@ -322,4 +339,96 @@ public List<String> listJars(String sessionId) {
         final SessionContext context = getSessionContext(sessionId);
         return context.listJars();
     }
+
+    @Override
+    public Optional<String> stopJob(
+            String sessionId, String jobId, boolean isWithSavepoint, boolean isWithDrain)
+            throws SqlExecutionException {
+        Duration clientTimeout = getSessionConfig(sessionId).get(ClientOptions.CLIENT_TIMEOUT);
+        try {
+            return runClusterAction(
+                    sessionId,
+                    clusterClient -> {
+                        if (isWithSavepoint) {
+                            // blocking get savepoint path
+                            try {
+                                String savepoint =
+                                        clusterClient
+                                                .stopWithSavepoint(
+                                                        JobID.fromHexString(jobId),
+                                                        isWithDrain,
+                                                        null,
+                                                        SavepointFormatType.DEFAULT)
+                                                .get(
+                                                        clientTimeout.toMillis(),
+                                                        TimeUnit.MILLISECONDS);
+                                return Optional.of(savepoint);
+                            } catch (Exception e) {
+                                throw new FlinkException(
+                                        "Could not stop job "
+                                                + jobId
+                                                + " in session "
+                                                + sessionId
+                                                + ".",
+                                        e);
+                            }
+                        } else {
+                            clusterClient.cancel(JobID.fromHexString(jobId));
+                            return Optional.empty();
+                        }
+                    });
+        } catch (Exception e) {
+            throw new SqlExecutionException(
+                    "Could not stop job " + jobId + " in session " + sessionId + ".", e);
+        }
+    }
+
+    /**
+     * Retrieves the {@link ClusterClient} from the session and runs the given {@link ClusterAction}
+     * against it.
+     *
+     * @param sessionId the specified session ID
+     * @param clusterAction the cluster action to run against the retrieved {@link ClusterClient}.
+     * @param <ClusterID> type of the cluster id
+     * @param <Result>> type of the result
+     * @throws FlinkException if something goes wrong
+     */
+    private <ClusterID, Result> Result runClusterAction(
+            String sessionId, ClusterAction<ClusterID, Result> clusterAction)
+            throws FlinkException {
+        final SessionContext context = getSessionContext(sessionId);
+        final Configuration configuration = (Configuration) context.getReadableConfig();
+        final ClusterClientFactory<ClusterID> clusterClientFactory =
+                clusterClientServiceLoader.getClusterClientFactory(configuration);
+
+        final ClusterID clusterId = clusterClientFactory.getClusterId(configuration);
+        Preconditions.checkNotNull(clusterId, "No cluster ID found for session " + sessionId);
+
+        try (final ClusterDescriptor<ClusterID> clusterDescriptor =
+                clusterClientFactory.createClusterDescriptor(configuration)) {
+            try (final ClusterClient<ClusterID> clusterClient =
+                    clusterDescriptor.retrieve(clusterId).getClusterClient()) {
+                return clusterAction.runAction(clusterClient);
+            }
+        }

Review Comment:
   Simplify:
   
   ```
   try (final ClusterDescriptor<ClusterID> clusterDescriptor =
                           clusterClientFactory.createClusterDescriptor(configuration);
                   final ClusterClient<ClusterID> clusterClient =
                           clusterDescriptor.retrieve(clusterId).getClusterClient()) {
               return clusterAction.runAction(clusterClient);
           }
   ```



##########
flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java:
##########
@@ -1683,6 +1683,11 @@ void testTryCast() {
                         "TRY_CAST(`A` AS ROW(`F0` INTEGER ARRAY, `F1` MAP< STRING, DECIMAL(10, 2) >, `F2` STRING NOT NULL))");
     }
 
+    @Test
+    void testStopJob() {
+        sql("STOP JOB 'myjob'").ok("STOP JOB 'myjob'");

Review Comment:
   Also, check `with savepoint` and `with drain`.



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java:
##########
@@ -115,6 +121,14 @@ private static Configuration getConfig() {
         config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
         config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
         config.setBoolean(WebOptions.SUBMIT_ENABLE, false);
+        config.set(StateBackendOptions.STATE_BACKEND, "hashmap");
+        config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
+        config.set(
+                CheckpointingOptions.CHECKPOINTS_DIRECTORY,
+                Paths.get(System.getProperty("java.io.tmpdir")).toUri().toString());
+        config.set(
+                CheckpointingOptions.SAVEPOINT_DIRECTORY,
+                Paths.get(System.getProperty("java.io.tmpdir")).toUri().toString());

Review Comment:
   Why not set the temporary folder path?



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java:
##########
@@ -406,6 +420,38 @@ public void testBatchQueryExecutionMultipleTimes() throws Exception {
         }
     }
 
+    @Test(timeout = 90_000L)
+    public void testStopJob() throws Exception {
+        final Map<String, String> configMap = new HashMap<>();
+        configMap.put(EXECUTION_RESULT_MODE.key(), ResultMode.TABLE.name());
+        configMap.put(RUNTIME_MODE.key(), RuntimeExecutionMode.STREAMING.name());
+        configMap.put(TableConfigOptions.TABLE_DML_SYNC.key(), "false");
+
+        final LocalExecutor executor =
+                createLocalExecutor(
+                        Collections.singletonList(udfDependency), Configuration.fromMap(configMap));
+        String sessionId = executor.openSession("test-session");
+
+        final String srcDdl = "CREATE TABLE src (a STRING) WITH ('connector' = 'datagen')";
+        final String snkDdl = "CREATE TABLE snk (a STRING) WITH ('connector' = 'blackhole')";
+        final String insert = "INSERT INTO snk SELECT a FROM src;";
+
+        try {
+            executor.executeOperation(sessionId, executor.parseStatement(sessionId, srcDdl));
+            executor.executeOperation(sessionId, executor.parseStatement(sessionId, snkDdl));
+            TableResult result =
+                    executor.executeOperation(
+                            sessionId, executor.parseStatement(sessionId, insert));
+            JobID jobId = result.getJobClient().get().getJobID();
+            // wait till the job turns into running status
+            Thread.sleep(2_000L);

Review Comment:
   We can use the job client to fetch the job status. 
   
   



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/tcl/SqlStopJob.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.sql.parser.tcl;

Review Comment:
   TCL means transaction control language. Actually, we don't offer any transaction-level semantics here. From my side, I think it's more like a DDL, which manages metadata of the running jobs. What do you think? 



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java:
##########
@@ -329,6 +332,68 @@ public void testCancelExecutionInteractiveMode() throws Exception {
         }
     }
 
+    @Test(timeout = 10000)
+    public void testStopJob() throws Exception {
+        final MockExecutor mockExecutor = new MockExecutor();
+        mockExecutor.isSync = false;
+
+        String sessionId = mockExecutor.openSession("test-session");
+        OutputStream outputStream = new ByteArrayOutputStream(256);
+        try (CliClient client =
+                new CliClient(
+                        () -> TerminalUtils.createDumbTerminal(outputStream),
+                        sessionId,
+                        mockExecutor,
+                        historyTempFile(),
+                        null)) {
+            client.executeInNonInteractiveMode(INSERT_INTO_STATEMENT);
+            String dmlResult = outputStream.toString();
+
+            Pattern pattern = Pattern.compile("[\\s\\S]*Job ID: (.*)[\\s\\S]*");
+            Matcher matcher = pattern.matcher(dmlResult);
+            assertThat(matcher.matches()).isTrue();
+            String jobId = matcher.group(1);

Review Comment:
   Add a method, e.g. getJobID to reuse these codes.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java:
##########
@@ -322,4 +339,96 @@ public List<String> listJars(String sessionId) {
         final SessionContext context = getSessionContext(sessionId);
         return context.listJars();
     }
+
+    @Override
+    public Optional<String> stopJob(
+            String sessionId, String jobId, boolean isWithSavepoint, boolean isWithDrain)
+            throws SqlExecutionException {
+        Duration clientTimeout = getSessionConfig(sessionId).get(ClientOptions.CLIENT_TIMEOUT);
+        try {
+            return runClusterAction(
+                    sessionId,
+                    clusterClient -> {
+                        if (isWithSavepoint) {
+                            // blocking get savepoint path
+                            try {
+                                String savepoint =
+                                        clusterClient
+                                                .stopWithSavepoint(
+                                                        JobID.fromHexString(jobId),
+                                                        isWithDrain,
+                                                        null,
+                                                        SavepointFormatType.DEFAULT)
+                                                .get(
+                                                        clientTimeout.toMillis(),
+                                                        TimeUnit.MILLISECONDS);
+                                return Optional.of(savepoint);
+                            } catch (Exception e) {
+                                throw new FlinkException(
+                                        "Could not stop job "
+                                                + jobId
+                                                + " in session "
+                                                + sessionId
+                                                + ".",
+                                        e);
+                            }
+                        } else {
+                            clusterClient.cancel(JobID.fromHexString(jobId));
+                            return Optional.empty();
+                        }
+                    });
+        } catch (Exception e) {
+            throw new SqlExecutionException(
+                    "Could not stop job " + jobId + " in session " + sessionId + ".", e);
+        }
+    }
+
+    /**
+     * Retrieves the {@link ClusterClient} from the session and runs the given {@link ClusterAction}
+     * against it.
+     *
+     * @param sessionId the specified session ID
+     * @param clusterAction the cluster action to run against the retrieved {@link ClusterClient}.
+     * @param <ClusterID> type of the cluster id
+     * @param <Result>> type of the result
+     * @throws FlinkException if something goes wrong
+     */
+    private <ClusterID, Result> Result runClusterAction(

Review Comment:
   It seems we plan to copy most of the codes from the `CliFrontEnd` about ClusterAction. I just think it's better to make the `ClusterAction` public and we can reuse these codes. WDYT?



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