yuzelin commented on code in PR #7402:
URL: https://github.com/apache/paimon/pull/7402#discussion_r2987855080
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java:
##########
@@ -324,7 +338,19 @@ void tryRefresh() throws Exception {
return;
}
- // 2. refresh dynamic partition
+ // 2. check if async partition refresh has completed, and switch if so
+ if (partitionRefresher != null &&
partitionRefresher.isPartitionRefreshAsync()) {
+ LookupTable switchedTable =
+ partitionRefresher.checkPartitionRefreshCompletion(
+ partitionLoader.partitions());
+ if (switchedTable != null) {
+ lookupTable.close();
+ lookupTable = switchedTable;
+ path = ((FullCacheLookupTable) switchedTable).context.tempPath;
Review Comment:
Handle path in more proper way.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java:
##########
@@ -236,6 +240,13 @@ private void open() throws Exception {
lookupTable.specifyPartitions(
partitions,
partitionLoader.createSpecificPartFilter());
}
+ if (partitionLoader instanceof DynamicPartitionLoader) {
+ // Initialize partition refresher
+ this.partitionRefresher =
+ new PartitionRefresher(
+
options.get(LOOKUP_DYNAMIC_PARTITION_REFRESH_ASYNC), table.name());
Review Comment:
options.get(LOOKUP_DYNAMIC_PARTITION_REFRESH_ASYNC)
&& lookupTable instance of FullCacheLookupTable
##########
paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java:
##########
@@ -1250,4 +1250,408 @@ public void testFallbackCacheMode() throws Exception {
iterator.close();
}
+
+ @ParameterizedTest
+ @EnumSource(
+ value = LookupCacheMode.class,
+ names = {"FULL"})
+ public void testSyncPartitionRefresh(LookupCacheMode mode) throws
Exception {
+ // This test verifies synchronous partition refresh (default mode):
+ // when max_pt() changes, the lookup table is refreshed synchronously,
+ // so queries immediately return new partition data after refresh.
+ sql(
+ "CREATE TABLE PARTITIONED_DIM (pt STRING, k INT, v INT,
PRIMARY KEY (pt, k) NOT ENFORCED)"
+ + "PARTITIONED BY (`pt`) WITH ("
+ + "'bucket' = '1', "
+ + "'lookup.dynamic-partition' = 'max_pt()', "
+ + "'lookup.dynamic-partition.refresh-interval' = '1
ms', "
+ + "'lookup.cache' = '%s', "
+ + "'continuous.discovery-interval'='1 ms')",
+ mode);
+
+ // insert data into partition '1'
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('1', 1, 100), ('1', 2, 200)");
+
+ String query =
+ "SELECT T.i, D.v FROM T LEFT JOIN PARTITIONED_DIM "
+ + "for system_time as of T.proctime AS D ON T.i = D.k";
+ BlockingIterator<Row, Row> iterator =
BlockingIterator.of(sEnv.executeSql(query).collect());
+
+ // verify initial lookup returns partition '1' data
+ sql("INSERT INTO T VALUES (1), (2)");
+ List<Row> result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 100), Row.of(2,
200));
+
+ // insert data into a new partition '2', which will trigger sync
partition refresh
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('2', 1, 1000), ('2', 2,
2000)");
+ sql("INSERT INTO T VALUES (1), (2)");
+ result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 1000),
Row.of(2, 2000));
+
+ iterator.close();
+ }
+
+ @ParameterizedTest
+ @EnumSource(
+ value = LookupCacheMode.class,
+ names = {"FULL", "MEMORY"})
+ public void testAsyncPartitionRefresh(LookupCacheMode mode) throws
Exception {
+ // This test verifies asynchronous partition refresh:
+ // when max_pt() changes, the lookup table is refreshed in a
background thread,
+ // old partition data continues serving queries until the new
partition is fully loaded.
+ sql(
+ "CREATE TABLE PARTITIONED_DIM (pt STRING, k INT, v INT,
PRIMARY KEY (pt, k) NOT ENFORCED)"
+ + "PARTITIONED BY (`pt`) WITH ("
+ + "'bucket' = '1', "
+ + "'lookup.dynamic-partition' = 'max_pt()', "
+ + "'lookup.dynamic-partition.refresh-interval' = '1
ms', "
+ + "'lookup.dynamic-partition.refresh.async' = 'true', "
+ + "'lookup.cache' = '%s', "
+ + "'continuous.discovery-interval'='1 ms')",
+ mode);
+
+ // insert data into partition '1'
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('1', 1, 100), ('1', 2, 200)");
+
+ String query =
+ "SELECT T.i, D.v FROM T LEFT JOIN PARTITIONED_DIM "
+ + "for system_time as of T.proctime AS D ON T.i = D.k";
+ BlockingIterator<Row, Row> iterator =
BlockingIterator.of(sEnv.executeSql(query).collect());
+
+ // verify initial lookup returns partition '1' data
+ sql("INSERT INTO T VALUES (1), (2)");
+ List<Row> result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 100), Row.of(2,
200));
+
+ // insert data into a new partition '2', which will trigger async
partition refresh
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('2', 1, 1000), ('2', 2,
2000)");
+ Thread.sleep(500); // wait for async refresh to complete
+ // trigger a lookup to check async completion and switch to new
partition
+ sql("INSERT INTO T VALUES (1), (2)");
+ iterator.collect(2);
+ Thread.sleep(500);
+ sql("INSERT INTO T VALUES (1), (2)");
+ result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 1000),
Row.of(2, 2000));
+
+ // insert another new partition '3' and verify switch again
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('3', 1, 10000), ('3', 2,
20000)");
+ Thread.sleep(500); // wait for async refresh to complete
+ sql("INSERT INTO T VALUES (1), (2)");
+ iterator.collect(2);
+ Thread.sleep(500);
+ sql("INSERT INTO T VALUES (1), (2)");
+ result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 10000),
Row.of(2, 20000));
+
+ iterator.close();
+ }
+
+ @ParameterizedTest
+ @EnumSource(
+ value = LookupCacheMode.class,
+ names = {"FULL", "MEMORY"})
+ public void
testAsyncPartitionRefreshServesOldDataDuringRefresh(LookupCacheMode mode)
+ throws Exception {
+ // Verify that during async refresh, queries still return old
partition data
+ // until the new partition is fully loaded and switched.
+ sql(
+ "CREATE TABLE PARTITIONED_DIM (pt STRING, k INT, v INT,
PRIMARY KEY (pt, k) NOT ENFORCED)"
+ + "PARTITIONED BY (`pt`) WITH ("
+ + "'bucket' = '1', "
+ + "'lookup.dynamic-partition' = 'max_pt()', "
+ + "'lookup.dynamic-partition.refresh-interval' = '1
ms', "
+ + "'lookup.dynamic-partition.refresh.async' = 'true', "
+ + "'lookup.cache' = '%s', "
+ + "'continuous.discovery-interval'='1 ms')",
+ mode);
+
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('1', 1, 100), ('1', 2, 200)");
+
+ String query =
+ "SELECT T.i, D.v FROM T LEFT JOIN PARTITIONED_DIM "
+ + "for system_time as of T.proctime AS D ON T.i = D.k";
+ BlockingIterator<Row, Row> iterator =
BlockingIterator.of(sEnv.executeSql(query).collect());
+
+ sql("INSERT INTO T VALUES (1), (2)");
+ List<Row> result = iterator.collect(2);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 100), Row.of(2,
200));
+
+ // insert new partition '2' to trigger async refresh
+ sql("INSERT INTO PARTITIONED_DIM VALUES ('2', 1, 1000), ('2', 2,
2000)");
+
+ // immediately query before async refresh completes — should still
return old partition data
+ sql("INSERT INTO T VALUES (1), (2)");
+ result = iterator.collect(2);
+ // old partition data (100, 200) should still be served
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 100), Row.of(2,
200));
+
+ // now wait for async refresh to complete and trigger switch
+ Thread.sleep(500);
+ sql("INSERT INTO T VALUES (1), (2)");
+ result = iterator.collect(2);
+ // after switch, new partition data should be returned
+ assertThat(result).containsExactlyInAnyOrder(Row.of(1, 1000),
Row.of(2, 2000));
+
+ iterator.close();
+ }
+
+ @ParameterizedTest
+ @EnumSource(
+ value = LookupCacheMode.class,
+ names = {"FULL", "MEMORY"})
+ public void
testAsyncPartitionRefreshWithDataUpdateInOldPartition(LookupCacheMode mode)
Review Comment:
Not needed.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
Review Comment:
move to constructor.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
+ this.scanPartitions = initialPartitions;
+ if (!partitionRefreshAsync) {
+ return;
+ }
+ this.pendingLookupTable = new AtomicReference<>(null);
+ this.partitionRefreshException = new AtomicReference<>(null);
+ this.partitionRefreshExecutor =
+ Executors.newSingleThreadExecutor(
+ new ExecutorThreadFactory(
+ String.format(
+ "%s-lookup-refresh-partition",
+ Thread.currentThread().getName())));
+ }
+
+ /** Get the current partitions being used for lookup. */
+ public List<BinaryRow> getScanPartitions() {
Review Comment:
public List<BinaryRow> getCurrentPartitions()
##########
paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java:
##########
@@ -1250,4 +1250,408 @@ public void testFallbackCacheMode() throws Exception {
iterator.close();
}
+
+ @ParameterizedTest
+ @EnumSource(
+ value = LookupCacheMode.class,
+ names = {"FULL"})
+ public void testSyncPartitionRefresh(LookupCacheMode mode) throws
Exception {
Review Comment:
Not needed.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
+ this.scanPartitions = initialPartitions;
+ if (!partitionRefreshAsync) {
+ return;
+ }
+ this.pendingLookupTable = new AtomicReference<>(null);
+ this.partitionRefreshException = new AtomicReference<>(null);
+ this.partitionRefreshExecutor =
+ Executors.newSingleThreadExecutor(
+ new ExecutorThreadFactory(
+ String.format(
+ "%s-lookup-refresh-partition",
+ Thread.currentThread().getName())));
+ }
+
+ /** Get the current partitions being used for lookup. */
+ public List<BinaryRow> getScanPartitions() {
+ return scanPartitions;
+ }
+
+ /**
+ * Start partition refresh. Chooses sync or async mode based on
configuration.
+ *
+ * @param newPartitions the new partitions to refresh to
+ * @param partitionFilter the partition filter for the new partitions
+ * @param lookupTable the current lookup table to refresh
+ * @param cacheRowFilter the cache row filter, may be null
+ */
+ public void startPartitionRefresh(
Review Comment:
startRefresh
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
+ this.scanPartitions = initialPartitions;
+ if (!partitionRefreshAsync) {
+ return;
+ }
+ this.pendingLookupTable = new AtomicReference<>(null);
+ this.partitionRefreshException = new AtomicReference<>(null);
+ this.partitionRefreshExecutor =
+ Executors.newSingleThreadExecutor(
+ new ExecutorThreadFactory(
+ String.format(
+ "%s-lookup-refresh-partition",
+ Thread.currentThread().getName())));
+ }
+
+ /** Get the current partitions being used for lookup. */
+ public List<BinaryRow> getScanPartitions() {
+ return scanPartitions;
+ }
+
+ /**
+ * Start partition refresh. Chooses sync or async mode based on
configuration.
+ *
+ * @param newPartitions the new partitions to refresh to
+ * @param partitionFilter the partition filter for the new partitions
+ * @param lookupTable the current lookup table to refresh
+ * @param cacheRowFilter the cache row filter, may be null
+ */
+ public void startPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable,
+ @Nullable Filter<InternalRow> cacheRowFilter)
+ throws Exception {
+ if (partitionRefreshAsync) {
+ asyncPartitionRefresh(
+ newPartitions,
+ partitionFilter,
+ ((FullCacheLookupTable) lookupTable).context,
+ cacheRowFilter);
+ } else {
+ syncPartitionRefresh(newPartitions, partitionFilter, lookupTable);
+ }
+ }
+
+ private void syncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable)
+ throws Exception {
+ LOG.info(
+ "Synchronously refreshing partition for table {}, new
partitions detected.",
+ tableName);
+ lookupTable.close();
+ lookupTable.specifyPartitions(newPartitions, partitionFilter);
+ lookupTable.open();
+ this.scanPartitions = newPartitions;
+ LOG.info("Synchronous partition refresh completed for table {}.",
tableName);
+ }
+
+ private void asyncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ FullCacheLookupTable.Context context,
+ @Nullable Filter<InternalRow> cacheRowFilter) {
+
+ LOG.info(
+ "Starting async partition refresh for table {}, new partitions
detected.",
+ tableName);
+
+ partitionRefreshExecutor.submit(
+ () -> {
+ File newPath = null;
+ try {
+ newPath =
+ new File(
+ context.tempPath.getParent(),
+ "lookup-" + UUID.randomUUID());
+ if (!newPath.mkdirs()) {
+ throw new RuntimeException("Failed to create dir:
" + newPath);
+ }
+ LookupTable newTable = copyWithNewPath(newPath,
context, cacheRowFilter);
+ newTable.specifyPartitions(newPartitions,
partitionFilter);
+ newTable.open();
+
+ pendingLookupTable.set(newTable);
+ LOG.info("Async partition refresh completed for table
{}.", tableName);
+ } catch (Exception e) {
+ LOG.error("Async partition refresh failed for table
{}.", tableName, e);
+ partitionRefreshException.set(e);
+ if (newPath != null) {
+ FileIOUtils.deleteDirectoryQuietly(newPath);
+ }
+ }
+ });
+ }
+
+ /**
+ * Create a new LookupTable instance with the same configuration but a
different temp path.
+ *
+ * @param newPath the new temp path
+ * @param context the context of the current lookup table
+ * @param cacheRowFilter the cache row filter, may be null
+ * @return a new LookupTable instance (not yet opened)
+ */
+ public LookupTable copyWithNewPath(
Review Comment:
And pass tempPathDirectory from FileStoreLookupFunction.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
+ this.scanPartitions = initialPartitions;
+ if (!partitionRefreshAsync) {
+ return;
+ }
+ this.pendingLookupTable = new AtomicReference<>(null);
+ this.partitionRefreshException = new AtomicReference<>(null);
+ this.partitionRefreshExecutor =
+ Executors.newSingleThreadExecutor(
+ new ExecutorThreadFactory(
+ String.format(
+ "%s-lookup-refresh-partition",
+ Thread.currentThread().getName())));
+ }
+
+ /** Get the current partitions being used for lookup. */
+ public List<BinaryRow> getScanPartitions() {
+ return scanPartitions;
+ }
+
+ /**
+ * Start partition refresh. Chooses sync or async mode based on
configuration.
+ *
+ * @param newPartitions the new partitions to refresh to
+ * @param partitionFilter the partition filter for the new partitions
+ * @param lookupTable the current lookup table to refresh
+ * @param cacheRowFilter the cache row filter, may be null
+ */
+ public void startPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable,
+ @Nullable Filter<InternalRow> cacheRowFilter)
+ throws Exception {
+ if (partitionRefreshAsync) {
+ asyncPartitionRefresh(
+ newPartitions,
+ partitionFilter,
+ ((FullCacheLookupTable) lookupTable).context,
+ cacheRowFilter);
+ } else {
+ syncPartitionRefresh(newPartitions, partitionFilter, lookupTable);
+ }
+ }
+
+ private void syncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable)
+ throws Exception {
+ LOG.info(
+ "Synchronously refreshing partition for table {}, new
partitions detected.",
+ tableName);
+ lookupTable.close();
+ lookupTable.specifyPartitions(newPartitions, partitionFilter);
+ lookupTable.open();
+ this.scanPartitions = newPartitions;
+ LOG.info("Synchronous partition refresh completed for table {}.",
tableName);
+ }
+
+ private void asyncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ FullCacheLookupTable.Context context,
+ @Nullable Filter<InternalRow> cacheRowFilter) {
+
+ LOG.info(
+ "Starting async partition refresh for table {}, new partitions
detected.",
+ tableName);
+
+ partitionRefreshExecutor.submit(
+ () -> {
+ File newPath = null;
+ try {
+ newPath =
+ new File(
+ context.tempPath.getParent(),
+ "lookup-" + UUID.randomUUID());
+ if (!newPath.mkdirs()) {
+ throw new RuntimeException("Failed to create dir:
" + newPath);
+ }
+ LookupTable newTable = copyWithNewPath(newPath,
context, cacheRowFilter);
+ newTable.specifyPartitions(newPartitions,
partitionFilter);
+ newTable.open();
+
+ pendingLookupTable.set(newTable);
+ LOG.info("Async partition refresh completed for table
{}.", tableName);
+ } catch (Exception e) {
+ LOG.error("Async partition refresh failed for table
{}.", tableName, e);
+ partitionRefreshException.set(e);
+ if (newPath != null) {
+ FileIOUtils.deleteDirectoryQuietly(newPath);
+ }
+ }
+ });
+ }
+
+ /**
+ * Create a new LookupTable instance with the same configuration but a
different temp path.
+ *
+ * @param newPath the new temp path
+ * @param context the context of the current lookup table
+ * @param cacheRowFilter the cache row filter, may be null
+ * @return a new LookupTable instance (not yet opened)
+ */
+ public LookupTable copyWithNewPath(
Review Comment:
private
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionRefresher.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package org.apache.paimon.flink.lookup;
+
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.ExecutorThreadFactory;
+import org.apache.paimon.utils.ExecutorUtils;
+import org.apache.paimon.utils.FileIOUtils;
+import org.apache.paimon.utils.Filter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static
org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS;
+
+/** Manages partition refresh logic for {@link FullCacheLookupTable}. */
+public class PartitionRefresher implements Closeable {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(PartitionRefresher.class);
+
+ private final boolean partitionRefreshAsync;
+ private final String tableName;
+
+ @Nullable private ExecutorService partitionRefreshExecutor;
+ private AtomicReference<LookupTable> pendingLookupTable;
+ private AtomicReference<Exception> partitionRefreshException;
+
+ /** Current partitions being used for lookup. Updated when partition
refresh completes. */
+ private List<BinaryRow> scanPartitions;
+
+ public PartitionRefresher(boolean partitionRefreshAsync, String tableName)
{
+ this.partitionRefreshAsync = partitionRefreshAsync;
+ this.tableName = tableName;
+ }
+
+ /**
+ * Initialize partition refresh resources. Should be called during table
initialization.
+ *
+ * @param initialPartitions the initial partitions to use for lookup
+ */
+ public void init(List<BinaryRow> initialPartitions) {
+ this.scanPartitions = initialPartitions;
+ if (!partitionRefreshAsync) {
+ return;
+ }
+ this.pendingLookupTable = new AtomicReference<>(null);
+ this.partitionRefreshException = new AtomicReference<>(null);
+ this.partitionRefreshExecutor =
+ Executors.newSingleThreadExecutor(
+ new ExecutorThreadFactory(
+ String.format(
+ "%s-lookup-refresh-partition",
+ Thread.currentThread().getName())));
+ }
+
+ /** Get the current partitions being used for lookup. */
+ public List<BinaryRow> getScanPartitions() {
+ return scanPartitions;
+ }
+
+ /**
+ * Start partition refresh. Chooses sync or async mode based on
configuration.
+ *
+ * @param newPartitions the new partitions to refresh to
+ * @param partitionFilter the partition filter for the new partitions
+ * @param lookupTable the current lookup table to refresh
+ * @param cacheRowFilter the cache row filter, may be null
+ */
+ public void startPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable,
+ @Nullable Filter<InternalRow> cacheRowFilter)
+ throws Exception {
+ if (partitionRefreshAsync) {
+ asyncPartitionRefresh(
+ newPartitions,
+ partitionFilter,
+ ((FullCacheLookupTable) lookupTable).context,
+ cacheRowFilter);
+ } else {
+ syncPartitionRefresh(newPartitions, partitionFilter, lookupTable);
+ }
+ }
+
+ private void syncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ LookupTable lookupTable)
+ throws Exception {
+ LOG.info(
+ "Synchronously refreshing partition for table {}, new
partitions detected.",
+ tableName);
+ lookupTable.close();
+ lookupTable.specifyPartitions(newPartitions, partitionFilter);
+ lookupTable.open();
+ this.scanPartitions = newPartitions;
+ LOG.info("Synchronous partition refresh completed for table {}.",
tableName);
+ }
+
+ private void asyncPartitionRefresh(
+ List<BinaryRow> newPartitions,
+ @Nullable Predicate partitionFilter,
+ FullCacheLookupTable.Context context,
+ @Nullable Filter<InternalRow> cacheRowFilter) {
+
+ LOG.info(
+ "Starting async partition refresh for table {}, new partitions
detected.",
+ tableName);
+
+ partitionRefreshExecutor.submit(
+ () -> {
+ File newPath = null;
+ try {
+ newPath =
+ new File(
+ context.tempPath.getParent(),
+ "lookup-" + UUID.randomUUID());
+ if (!newPath.mkdirs()) {
+ throw new RuntimeException("Failed to create dir:
" + newPath);
+ }
+ LookupTable newTable = copyWithNewPath(newPath,
context, cacheRowFilter);
+ newTable.specifyPartitions(newPartitions,
partitionFilter);
+ newTable.open();
+
+ pendingLookupTable.set(newTable);
+ LOG.info("Async partition refresh completed for table
{}.", tableName);
+ } catch (Exception e) {
+ LOG.error("Async partition refresh failed for table
{}.", tableName, e);
+ partitionRefreshException.set(e);
+ if (newPath != null) {
+ FileIOUtils.deleteDirectoryQuietly(newPath);
+ }
+ }
+ });
+ }
+
+ /**
+ * Create a new LookupTable instance with the same configuration but a
different temp path.
+ *
+ * @param newPath the new temp path
+ * @param context the context of the current lookup table
+ * @param cacheRowFilter the cache row filter, may be null
+ * @return a new LookupTable instance (not yet opened)
+ */
+ public LookupTable copyWithNewPath(
+ File newPath,
+ FullCacheLookupTable.Context context,
+ @Nullable Filter<InternalRow> cacheRowFilter) {
+ FullCacheLookupTable.Context newContext = context.copy(newPath);
+ Options options = Options.fromMap(context.table.options());
+ FullCacheLookupTable newTable =
+ FullCacheLookupTable.create(newContext,
options.get(LOOKUP_CACHE_ROWS));
+ if (cacheRowFilter != null) {
+ newTable.specifyCacheRowFilter(cacheRowFilter);
+ }
+ return newTable;
+ }
+
+ /**
+ * Check if an async partition refresh has completed.
+ *
+ * @param newPartitions the new partitions to update after refresh
completes
+ * @return the new lookup table if ready, or null if no switch is needed
+ */
+ @Nullable
+ public LookupTable checkPartitionRefreshCompletion(List<BinaryRow>
newPartitions)
Review Comment:
getNewLookupTable
--
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]