4343import org .apache .paimon .types .RowType ;
4444import org .apache .paimon .utils .TraceableFileIO ;
4545
46+ import org .apache .flink .table .data .RowData ;
4647import org .junit .jupiter .api .AfterEach ;
4748import org .junit .jupiter .api .BeforeEach ;
4849import org .junit .jupiter .api .Test ;
@@ -91,16 +92,19 @@ public void before() throws Exception {
9192 }
9293
9394 private void createLookupFunction (boolean refreshAsync ) throws Exception {
94- createLookupFunction (true , false , false , refreshAsync );
95+ createLookupFunction (true , false , false , refreshAsync , null );
9596 }
9697
9798 private void createLookupFunction (
9899 boolean isPartition ,
99100 boolean joinEqualPk ,
100101 boolean dynamicPartition ,
101- boolean refreshAsync )
102+ boolean refreshAsync ,
103+ Integer fullLoadThreshold )
102104 throws Exception {
103- table = createFileStoreTable (isPartition , dynamicPartition , refreshAsync );
105+ table =
106+ createFileStoreTable (
107+ isPartition , dynamicPartition , refreshAsync , fullLoadThreshold );
104108 lookupFunction = createLookupFunction (table , joinEqualPk );
105109 lookupFunction .open (tempDir .toString ());
106110 }
@@ -116,7 +120,11 @@ private FileStoreLookupFunction createLookupFunction(
116120 }
117121
118122 private FileStoreTable createFileStoreTable (
119- boolean isPartition , boolean dynamicPartition , boolean refreshAsync ) throws Exception {
123+ boolean isPartition ,
124+ boolean dynamicPartition ,
125+ boolean refreshAsync ,
126+ Integer fullLoadThreshold )
127+ throws Exception {
120128 SchemaManager schemaManager = new SchemaManager (fileIO , tablePath );
121129 Options conf = new Options ();
122130 conf .set (FlinkConnectorOptions .LOOKUP_REFRESH_ASYNC , refreshAsync );
@@ -128,6 +136,10 @@ private FileStoreTable createFileStoreTable(
128136 conf .set (FlinkConnectorOptions .SCAN_PARTITIONS , "max_pt()" );
129137 }
130138
139+ if (fullLoadThreshold != null ) {
140+ conf .set (FlinkConnectorOptions .LOOKUP_REFRESH_FULL_LOAD_THRESHOLD , fullLoadThreshold );
141+ }
142+
131143 RowType rowType =
132144 RowType .of (
133145 new DataType [] {DataTypes .INT (), DataTypes .INT (), DataTypes .BIGINT ()},
@@ -153,7 +165,7 @@ public void close() throws Exception {
153165
154166 @ Test
155167 public void testCompatibilityForOldVersion () throws Exception {
156- createLookupFunction (false , true , false , false );
168+ createLookupFunction (false , true , false , false , null );
157169 commit (writeCommit (1 ));
158170 PrimaryKeyPartialLookupTable lookupTable =
159171 (PrimaryKeyPartialLookupTable ) lookupFunction .lookupTable ();
@@ -174,7 +186,7 @@ public void testCompatibilityForOldVersion() throws Exception {
174186 @ ParameterizedTest
175187 @ ValueSource (booleans = {false , true })
176188 public void testDefaultLocalPartial (boolean refreshAsync ) throws Exception {
177- createLookupFunction (false , true , false , refreshAsync );
189+ createLookupFunction (false , true , false , refreshAsync , null );
178190 assertThat (lookupFunction .lookupTable ()).isInstanceOf (PrimaryKeyPartialLookupTable .class );
179191 QueryExecutor queryExecutor =
180192 ((PrimaryKeyPartialLookupTable ) lookupFunction .lookupTable ()).queryExecutor ();
@@ -184,7 +196,7 @@ public void testDefaultLocalPartial(boolean refreshAsync) throws Exception {
184196 @ ParameterizedTest
185197 @ ValueSource (booleans = {false , true })
186198 public void testDefaultRemotePartial (boolean refreshAsync ) throws Exception {
187- createLookupFunction (false , true , false , refreshAsync );
199+ createLookupFunction (false , true , false , refreshAsync , null );
188200 ServiceManager serviceManager = new ServiceManager (fileIO , tablePath );
189201 serviceManager .resetService (
190202 PRIMARY_KEY_LOOKUP , new InetSocketAddress [] {new InetSocketAddress (1 )});
@@ -232,7 +244,7 @@ public void testLookupExpiredSnapshot(boolean refreshAsync) throws Exception {
232244
233245 @ Test
234246 public void testLookupDynamicPartition () throws Exception {
235- createLookupFunction (true , false , true , false );
247+ createLookupFunction (true , false , true , false , null );
236248 commit (writeCommit (1 ));
237249 lookupFunction .lookup (new FlinkRowData (GenericRow .of (1 , 1 , 10L )));
238250 assertThat (
@@ -252,7 +264,7 @@ public void testLookupDynamicPartition() throws Exception {
252264
253265 @ Test
254266 public void testParseWrongTimePeriodsBlacklist () throws Exception {
255- FileStoreTable table = createFileStoreTable (false , false , false );
267+ FileStoreTable table = createFileStoreTable (false , false , false , null );
256268
257269 FileStoreTable table1 =
258270 table .copy (
@@ -299,7 +311,7 @@ public void testCheckRefreshInBlacklist() throws Exception {
299311 String right = end .atZone (ZoneId .systemDefault ()).format (formatter );
300312
301313 FileStoreTable table =
302- createFileStoreTable (false , false , false )
314+ createFileStoreTable (false , false , false , null )
303315 .copy (
304316 Collections .singletonMap (
305317 LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST .key (),
@@ -312,6 +324,50 @@ public void testCheckRefreshInBlacklist() throws Exception {
312324 assertThat (lookupFunction .nextBlacklistCheckTime ()).isEqualTo (end .toEpochMilli () + 1 );
313325 }
314326
327+ @ ParameterizedTest
328+ @ ValueSource (booleans = {false , true })
329+ public void testLookupTableWithFullLoad (boolean joinEqualPk ) throws Exception {
330+ createLookupFunction (false , joinEqualPk , false , false , 3 );
331+
332+ if (joinEqualPk ) {
333+ assertThat (lookupFunction .lookupTable ())
334+ .isInstanceOf (PrimaryKeyPartialLookupTable .class );
335+ } else {
336+ assertThat (lookupFunction .lookupTable ()).isInstanceOf (FullCacheLookupTable .class );
337+ }
338+
339+ GenericRow expectedRow = GenericRow .of (1 , 1 , 1L );
340+ StreamTableWrite writer = table .newStreamWriteBuilder ().newWrite ();
341+ writer .write (expectedRow );
342+ commit (writer .prepareCommit (true , 1 ));
343+
344+ List <RowData > result =
345+ new ArrayList <>(lookupFunction .lookup (new FlinkRowData (GenericRow .of (1 , 1 , 1L ))));
346+ assertThat (result ).size ().isEqualTo (1 );
347+ RowData resultRow = result .get (0 );
348+ assertThat (resultRow .getInt (0 )).isEqualTo (expectedRow .getInt (0 ));
349+ assertThat (resultRow .getInt (1 )).isEqualTo (expectedRow .getInt (1 ));
350+
351+ // Create more commits to exceed threshold (3 more to have gap > 3)
352+ for (int i = 2 ; i < 6 ; i ++) {
353+ writer .write (GenericRow .of (i , i , (long ) i ));
354+ commit (writer .prepareCommit (true , i ));
355+ }
356+ writer .close ();
357+
358+ // wait refresh
359+ Thread .sleep (2000 );
360+
361+ expectedRow = GenericRow .of (5 , 5 , 5L );
362+ assertThat (lookupFunction .shouldDoFullLoad ()).isTrue ();
363+ lookupFunction .tryRefresh ();
364+ result = new ArrayList <>(lookupFunction .lookup (new FlinkRowData (GenericRow .of (5 , 5 , 5L ))));
365+ assertThat (result ).size ().isEqualTo (1 );
366+ resultRow = result .get (0 );
367+ assertThat (resultRow .getInt (0 )).isEqualTo (expectedRow .getInt (0 ));
368+ assertThat (resultRow .getInt (1 )).isEqualTo (expectedRow .getInt (1 ));
369+ }
370+
315371 private void commit (List <CommitMessage > messages ) throws Exception {
316372 TableCommitImpl commit = table .newCommit (commitUser );
317373 commit .commit (messages );
0 commit comments