This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new e807ff8f9 [bug] Continuously refresh while invoking look-up join 
(#1364)
e807ff8f9 is described below

commit e807ff8f9d7da5c837a1b312f91e9b9a50c87639
Author: YeJunHao <[email protected]>
AuthorDate: Tue Jun 13 22:13:35 2023 +0800

    [bug] Continuously refresh while invoking look-up join (#1364)
---
 .../org/apache/paimon/flink/lookup/FileStoreLookupFunction.java     | 6 +++---
 .../java/org/apache/paimon/flink/lookup/TableStreamingReader.java   | 2 +-
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
index c9e4e47c9..68a1d225f 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
@@ -194,9 +194,9 @@ public class FileStoreLookupFunction implements 
Serializable, Closeable {
     }
 
     private void refresh() throws Exception {
-        try (RecordReaderIterator<InternalRow> batch =
-                new RecordReaderIterator<>(streamingReader.getRecordReader())) 
{
-            while (true) {
+        while (true) {
+            try (RecordReaderIterator<InternalRow> batch =
+                    new RecordReaderIterator<>(streamingReader.nextBatch())) {
                 if (!batch.hasNext()) {
                     return;
                 }
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/TableStreamingReader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/TableStreamingReader.java
index b66a89c96..1582a384e 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/TableStreamingReader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/TableStreamingReader.java
@@ -92,7 +92,7 @@ public class TableStreamingReader {
         }
     }
 
-    public RecordReader<InternalRow> getRecordReader() throws Exception {
+    public RecordReader<InternalRow> nextBatch() throws Exception {
         try {
             return read(scan.plan());
         } catch (EndOfScanException e) {

Reply via email to