voonhous commented on code in PR #8711:
URL: https://github.com/apache/hudi/pull/8711#discussion_r1194939293
##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java:
##########
@@ -303,22 +316,43 @@ private void checkAnswerWithMeta(TableOptions
tableOptions, String... expectedRe
+ " last_name string,"
+ " salary double,"
+ " ts timestamp,"
+ + " f_struct row<f0 int, f2 int, f1 string, f3 string>,"
+ " `partition` string"
+ ") partitioned by (`partition`) with (" + tableOptions + ")"
);
//language=SQL
- checkAnswer("select `_hoodie_record_key`, first_name, salary from t1",
expectedResult);
+ checkAnswer("select `_hoodie_record_key`, first_name, salary, f_struct
from t1", expectedResult);
}
- private void checkAnswer(String query, String... expectedResult) throws
Exception {
+ private void checkAnswer(String query, String... expectedResult) {
TableResult actualResult = tEnv.executeSql(query);
Set<String> expected = new HashSet<>(Arrays.asList(expectedResult));
- Set<String> actual = new HashSet<>(expected.size());
- try (CloseableIterator<Row> iterator = actualResult.collect()) {
- for (int i = 0; i < expected.size() && iterator.hasNext(); i++) {
- actual.add(iterator.next().toString());
+ Set<String> actual = new HashSet<>();
+
+ // create a runnable to handle reads (especially useful for streaming
reads as they are unbounded)
+ Runnable runnable = () -> {
+ try (CloseableIterator<Row> iterator = actualResult.collect()) {
+ while(iterator.hasNext()) {
+ actual.add(iterator.next().toString());
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
}
+ };
+
+ ExecutorService executor = Executors.newSingleThreadExecutor();
Review Comment:
https://github.com/apache/hudi/blob/a9b183c71ec4b96012294afd42bca677fa614a82/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java#L89-L95C4
Erm, this test is a streaming read, the iterator will keep blocking. It will
not exit...
--
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]