Skip to content

Commit

Permalink
[FLINK-21669][table-api] Support "table.dml-sync" option to execute s…
Browse files Browse the repository at this point in the history
…tatement in sync mode

This closes apache#15340
  • Loading branch information
fsk119 authored and wuchong committed Mar 28, 2021
1 parent 42c6414 commit 4782318
Show file tree
Hide file tree
Showing 12 changed files with 307 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,12 @@
</tr>
</thead>
<tbody>
<tr>
<td><h5>table.dml-sync</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Specifies if the DML job (i.e. the insert operation) is executed asynchronously or synchronously. By default, the execution is async, so you can submit multiple DML jobs at the same time. If set this option to true, the insert operation will wait for the job to finish.</td>
</tr>
<tr>
<td><h5>table.dynamic-table-options.enabled</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td>
<td style="word-wrap: break-word;">false</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,12 +60,16 @@
import java.util.Map;
import java.util.Optional;

import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
import static org.apache.flink.table.api.internal.TableResultImpl.TABLE_RESULT_OK;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_DEPRECATED_KEY;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_EXECUTE_STATEMENT;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_FINISH_STATEMENT;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_REMOVED_KEY;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_RESET_KEY;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_SET_KEY;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_STATEMENT_SUBMITTED;
import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_WAIT_EXECUTE;
import static org.apache.flink.table.client.config.YamlConfigUtils.getOptionNameWithDeprecatedKey;
import static org.apache.flink.table.client.config.YamlConfigUtils.getPropertiesInPretty;
import static org.apache.flink.table.client.config.YamlConfigUtils.isDeprecatedKey;
Expand Down Expand Up @@ -414,25 +418,33 @@ private void callSelect(QueryOperation operation) {
private boolean callInsert(CatalogSinkModifyOperation operation) {
printInfo(CliStrings.MESSAGE_SUBMITTING_STATEMENT);

TableResult tableResult = null;
boolean sync = executor.getSessionConfig(sessionId).get(TABLE_DML_SYNC);
if (sync) {
printInfo(MESSAGE_WAIT_EXECUTE);
}
try {
TableResult result = executor.executeOperation(sessionId, operation);
checkState(result.getJobClient().isPresent());
terminal.writer()
.println(
CliStrings.messageInfo(CliStrings.MESSAGE_STATEMENT_SUBMITTED)
.toAnsi());
// keep compatibility with before
terminal.writer()
.println(
String.format(
"Job ID: %s\n",
result.getJobClient().get().getJobID().toString()));
tableResult = executor.executeOperation(sessionId, operation);
checkState(tableResult.getJobClient().isPresent());

if (sync) {
terminal.writer()
.println(CliStrings.messageInfo(MESSAGE_FINISH_STATEMENT).toAnsi());
} else {
terminal.writer()
.println(CliStrings.messageInfo(MESSAGE_STATEMENT_SUBMITTED).toAnsi());
terminal.writer()
.println(
String.format(
"Job ID: %s\n",
tableResult.getJobClient().get().getJobID().toString()));
}
terminal.flush();
return true;
} catch (SqlExecutionException e) {
} catch (Exception e) {
printExecutionException(e);
return false;
}
return false;
return true;
}

private void executeOperation(Operation operation) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,13 +193,19 @@ private CliStrings() {
public static final String MESSAGE_SUBMITTING_STATEMENT =
"Submitting SQL update statement to the cluster...";

public static final String MESSAGE_FINISH_STATEMENT =
"Complete execution of the SQL update statement.";

public static final String MESSAGE_STATEMENT_SUBMITTED =
"Table update statement has been successfully submitted to the cluster:";

public static final String MESSAGE_WILL_EXECUTE = "Executing the following statement:";

public static final String MESSAGE_UNSUPPORTED_SQL = "Unsupported SQL statement.";

public static final String MESSAGE_WAIT_EXECUTE =
"Execute statement in sync mode. Please wait for the execution finish...";

public static final String MESSAGE_EXECUTE_STATEMENT = "Execute statement succeed.";

// --------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,8 @@ public static void setup() throws IOException {
replaceVars.put(
"$VAR_JOBMANAGER_RPC_ADDRESS",
miniClusterResource.getClientConfiguration().get(ADDRESS));
replaceVars.put("$VAR_STREAMING_PATH", tempFolder.newFolder().toPath().toString());
replaceVars.put("$VAR_BATCH_PATH", tempFolder.newFolder().toPath().toString());
}

@Test
Expand Down Expand Up @@ -164,6 +166,7 @@ private List<Result> runSqlStatements(List<String> statements) throws IOExceptio
// -------------------------------------------------------------------------------------------

private static final String PROMOTE = "Flink SQL> ";
private static final String JOB_ID = "Job ID:";

enum Tag {
ERROR("\u001B[31;1m", "\u001B[0m", "!error"),
Expand Down Expand Up @@ -245,7 +248,12 @@ private static List<Result> normalizeOutput(String output) {
// remove the promote prefix
line = line.substring(PROMOTE.length());
}
contentLines.add(line);
// ignore the line begin with Job ID:
if (!line.startsWith(JOB_ID)) {
contentLines.add(line);
} else {
contentLines.add(JOB_ID);
}
}
} catch (IOException e) {
throw new RuntimeException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
import javax.annotation.Nullable;

import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
Expand All @@ -67,7 +68,9 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.FutureTask;

import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
Expand Down Expand Up @@ -130,6 +133,38 @@ public void testHistoryFile() throws Exception {
}
}

@Test
public void verifyCancelSubmitInSyncMode() throws Exception {
final MockExecutor mockExecutor = new MockExecutor(true);
String sessionId = mockExecutor.openSession("test-session");

ByteArrayOutputStream outputStream = new ByteArrayOutputStream(256);

try (CliClient client =
new CliClient(
TerminalUtils.createDummyTerminal(outputStream),
sessionId,
mockExecutor,
historyTempFile(),
null)) {
FutureTask<Boolean> task =
new FutureTask<>(() -> client.submitUpdate(INSERT_INTO_STATEMENT));
Thread thread = new Thread(task);
thread.start();

while (!mockExecutor.isAwait) {
Thread.sleep(10);
}

thread.interrupt();
assertFalse(task.get());
assertTrue(
outputStream
.toString()
.contains("java.lang.InterruptedException: sleep interrupted"));
}
}

// --------------------------------------------------------------------------------------------

private void verifyUpdateSubmission(
Expand Down Expand Up @@ -189,14 +224,26 @@ private static class MockExecutor implements Executor {

public boolean failExecution;

public boolean isSync = false;
public boolean isAwait = false;
public String receivedStatement;
public int receivedPosition;
private Configuration configuration = new Configuration();
private final Map<String, SessionContext> sessionMap = new HashMap<>();
private final SqlParserHelper helper = new SqlParserHelper();

@Override
public void start() throws SqlExecutionException {}

public MockExecutor() {
this(false);
}

public MockExecutor(boolean isSync) {
configuration.set(TABLE_DML_SYNC, isSync);
this.isSync = isSync;
}

@Override
public String openSession(@Nullable String sessionId) throws SqlExecutionException {
DefaultContext defaultContext =
Expand Down Expand Up @@ -247,6 +294,14 @@ public TableResult executeOperation(String sessionId, Operation operation)
throw new SqlExecutionException("Fail execution.");
}
if (operation instanceof ModifyOperation || operation instanceof QueryOperation) {
if (isSync) {
isAwait = true;
try {
Thread.sleep(Long.MAX_VALUE);
} catch (InterruptedException e) {
throw new SqlExecutionException("Fail to execute", e);
}
}
return new TestTableResult(
new TestingJobClient(),
ResultKind.SUCCESS_WITH_CONTENT,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ public Map<String, String> getSessionConfigMap(String sessionId) throws SqlExecu

@Override
public ReadableConfig getSessionConfig(String sessionId) throws SqlExecutionException {
return null;
throw new UnsupportedOperationException("Not implemented.");
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,9 @@ public Optional<JobClient> getJobClient() {
}

@Override
public void await() {}
public void await() throws InterruptedException {
Thread.sleep(60000);
}

@Override
public void await(long timeout, TimeUnit unit) {}
Expand Down
104 changes: 104 additions & 0 deletions flink-table/flink-sql-client/src/test/resources/sql/insert.q
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
# insert.q - insert
#
# 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.

SET sql-client.execution.result-mode = tableau;
[INFO] Session property has been set.
!info

SET table.dml-sync=true;
[INFO] Session property has been set.
!info

# ==========================================================================
# test streaming insert
# ==========================================================================

SET execution.runtime-mode = streaming;
[INFO] Session property has been set.
!info

create table StreamingTable (
id int,
str string
) with (
'connector' = 'filesystem',
'path' = '$VAR_STREAMING_PATH',
'format' = 'csv'
);
[INFO] Execute statement succeed.
!info

INSERT INTO StreamingTable SELECT * FROM (VALUES (1, 'Hello World'), (2, 'Hi'), (2, 'Hi'), (3, 'Hello'), (3, 'World'), (4, 'ADD'), (5, 'LINE'));
[INFO] Submitting SQL update statement to the cluster...
[INFO] Execute statement in sync mode. Please wait for the execution finish...
[INFO] Complete execution of the SQL update statement.
!info

SELECT * FROM StreamingTable;
+----+-------------+----------------------+
| op | id | str |
+----+-------------+----------------------+
| +I | 1 | Hello World |
| +I | 2 | Hi |
| +I | 2 | Hi |
| +I | 3 | Hello |
| +I | 3 | World |
| +I | 4 | ADD |
| +I | 5 | LINE |
+----+-------------+----------------------+
Received a total of 7 rows
!ok

# ==========================================================================
# test batch insert
# ==========================================================================

SET execution.runtime-mode = batch;
[INFO] Session property has been set.
!info

create table BatchTable (
id int,
str string
) with (
'connector' = 'filesystem',
'path' = '$VAR_BATCH_PATH',
'format' = 'csv'
);
[INFO] Execute statement succeed.
!info

INSERT INTO BatchTable SELECT * FROM (VALUES (1, 'Hello World'), (2, 'Hi'), (2, 'Hi'), (3, 'Hello'), (3, 'World'), (4, 'ADD'), (5, 'LINE'));
[INFO] Submitting SQL update statement to the cluster...
[INFO] Execute statement in sync mode. Please wait for the execution finish...
[INFO] Complete execution of the SQL update statement.
!info

SELECT * FROM BatchTable;
+-------------+----------------------+
| id | str |
+-------------+----------------------+
| 1 | Hello World |
| 2 | Hi |
| 2 | Hi |
| 3 | Hello |
| 3 | World |
| 4 | ADD |
| 5 | LINE |
+-------------+----------------------+
Received a total of 7 rows
!ok
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,16 @@ private TableConfigOptions() {}
+ "Note: The old planner will be removed in Flink 1.14, "
+ "so this option will become obsolete.");

@Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
public static final ConfigOption<Boolean> TABLE_DML_SYNC =
key("table.dml-sync")
.booleanType()
.defaultValue(false)
.withDescription(
"Specifies if the DML job (i.e. the insert operation) is executed asynchronously or synchronously. "
+ "By default, the execution is async, so you can submit multiple DML jobs at the same time. "
+ "If set this option to true, the insert operation will wait for the job to finish.");

@Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
public static final ConfigOption<Boolean> TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED =
key("table.dynamic-table-options.enabled")
Expand Down
Loading

0 comments on commit 4782318

Please sign in to comment.