-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-18337][table] Introduce TableResult#await method to wait for data ready #12688
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit a66d075 (Tue Jun 16 17:45:39 UTC 2020) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. DetailsThe Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
51d8c4c to
5faf259
Compare
|
cc @twalthr |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for this change @godfreyhe, it makes the API much nicer and concise. I left some comments.
| "FROM hTable AS h"); | ||
|
|
||
| List<Row> results = collectBatchResult(table); | ||
| List<Row> results = Lists.newArrayList(table.execute().collect()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
avoid dependencies to other modules for simple util functions. Introduce an overloaded method org.apache.flink.util.CollectionUtil#iteratorToList in a hotfix commit
| " h.family3.col2, " + | ||
| " h.family3.col3 " + | ||
| "FROM hTable AS h"); | ||
|
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For all locations that were updated in the previous commit the declared throws Exception is not necessary anymore. My IDE shows a lot of warnings Exception 'java.lang.Exception' is never thrown in the method which we can fix in this PR.
| "insert into db1.sink_table select 6,'a','b','2020-05-03','12'") | ||
| .await(); | ||
| } catch (Exception e) { | ||
| throw new TableException("Failed to execute sql", e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
throw AssertionError in tests with Assert.fail().
| """ | ||
| Wait if necessary for at most the given time (milliseconds) for the data to be ready. | ||
| For select operation, this method will wait unit the first row can be accessed in local. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For a select operation, this method will wait until the first row can be accessed locally.
For an insert operation, this method will wait for the job to finish, because the result contains only one row.
For other operations, this method will return immediately, because the result is already available locally.
| else: | ||
| return None | ||
|
|
||
| def wait(self, timeout_ms=None): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
call it await to be consistent. how are the exceptions handled in Python?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
await is a keyword in python.
just like as method in Table, python Table uses alias.
now flink-python does not define Exceptions, java exception will be thrown directly. example: python Catalog class
| * For insert operation, this method will wait for the job to finish, because the result contains only one row. | ||
| * For other operations, this method will return immediately, because the result is ready in local. | ||
| * | ||
| * @throws ExecutionException if this future completed exceptionally |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if this future completed exceptionally it is not really a future
| /** | ||
| * Iterator for insert operation result. | ||
| */ | ||
| private static final class InsertResultIterator implements CloseableIterator<Row> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TableEnvironmentImpl is already quite large. Move it to a separate class in default scope.
| } | ||
|
|
||
| ExecutorService executor = Executors.newFixedThreadPool( | ||
| 1, new ThreadFactoryBuilder().setNameFormat("TableResult-await-thread").build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
avoid guava
|
@twalthr Thanks for the suggestion, I have updated the pr |
twalthr
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the update @godfreyhe. I added my last batch of comments. Good to merge afterwards. Can you rebase the PR?
| import java.util.Optional; | ||
|
|
||
| /** | ||
| * A CloseableIterator for insert operation result. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: use {@link CloseableIterator} in such cases to make refactorings easier in the future
| class InsertResultIterator implements CloseableIterator<Row> { | ||
| private final JobClient jobClient; | ||
| private final Row affectedRowCountsRow; | ||
| private Optional<Boolean> hasNext = Optional.empty(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Optional should not be used in fields: https://flink.apache.org/contributing/code-style-and-quality-java.html#java-optional
| public boolean hasNext() { | ||
| if (!hasNext.isPresent()) { | ||
| try { | ||
| jobClient.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Make the classloader configurable for this class and pass the context classloader higher in the stack. The TableEnvironment will have it soon (see FLINK-15635).
| |GROUP BY TUMBLE(ts, INTERVAL '5' SECOND) | ||
| """.stripMargin | ||
| execInsertSqlAndWaitResult(tableEnv, query) | ||
| tableEnv.executeSql(query).await() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: wrong indention
…of guava Lists when collecting result
80045af to
9fc17e0
Compare
|
@twalthr I have updated the pr based on latest comments |
What is the purpose of the change
TableEnvironment.executeSql() method for INSERT statement returns TableResult once the job is submitted. Users must use tableResult.getJobClient.get().getJobExecutionResult(Thread.currentThread().getContextClassLoader).get() to wait the job finish. This API looks very ugly. This pr aims to introduce TableResult#await method to make the api fluent
Brief change log
Verifying this change
This change is already covered by existing tests*.
Does this pull request potentially affect one of the following parts:
@Public(Evolving): (yes / no)Documentation