Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/workflows/utitcase-flink-1.x-common.yml
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ concurrency:
jobs:
build_test:
runs-on: ubuntu-latest
timeout-minutes: 60
timeout-minutes: 100

steps:
- name: Checkout code
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,12 @@
<td>Duration</td>
<td>Specific dynamic partition refresh interval for lookup, scan all partitions and obtain corresponding partition.</td>
</tr>
<tr>
<td><h5>lookup.dynamic-partition.refresh.async</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Whether to refresh dynamic partition lookup table asynchronously. This option only works for full cache dimension table. When enabled, partition changes will be loaded in a background thread while the old partition data continues serving queries. When disabled (default), partition refresh is synchronous and blocks queries until the new partition data is fully loaded.</td>
</tr>
<tr>
<td><h5>lookup.refresh.async</h5></td>
<td style="word-wrap: break-word;">false</td>
Expand Down Expand Up @@ -357,4 +363,4 @@
<td>Defines a custom parallelism for the unaware-bucket table compaction job. By default, if this option is not defined, the planner will derive the parallelism for each statement individually by also considering the global configuration.</td>
</tr>
</tbody>
</table>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,18 @@ public class FlinkConnectorOptions {
.defaultValue(false)
.withDescription("Whether to refresh lookup table in an async thread.");

public static final ConfigOption<Boolean> LOOKUP_DYNAMIC_PARTITION_REFRESH_ASYNC =
ConfigOptions.key("lookup.dynamic-partition.refresh.async")
.booleanType()
.defaultValue(false)
.withDescription(
"Whether to refresh dynamic partition lookup table asynchronously. "
+ "This option only works for full cache dimension table. "
+ "When enabled, partition changes will be loaded in a background thread "
+ "while the old partition data continues serving queries. "
+ "When disabled (default), partition refresh is synchronous and blocks queries "
+ "until the new partition data is fully loaded.");

public static final ConfigOption<Integer> LOOKUP_REFRESH_ASYNC_PENDING_SNAPSHOT_COUNT =
ConfigOptions.key("lookup.refresh.async.pending-snapshot-count")
.intType()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@

import static org.apache.paimon.CoreOptions.CONTINUOUS_DISCOVERY_INTERVAL;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_DYNAMIC_PARTITION_REFRESH_ASYNC;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_FULL_LOAD_THRESHOLD;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST;
import static org.apache.paimon.flink.query.RemoteTableQuery.isRemoteServiceAvailable;
Expand All @@ -93,8 +94,12 @@ public class FileStoreLookupFunction implements Serializable, Closeable {
private final List<InternalRow.FieldGetter> projectFieldsGetters;

private transient File path;
private transient String tmpDirectory;
private transient LookupTable lookupTable;

// partition refresh
@Nullable private transient PartitionRefresher partitionRefresher;

// interval of refreshing lookup table
private transient Duration refreshInterval;
// timestamp when refreshing lookup table
Expand Down Expand Up @@ -161,7 +166,7 @@ public FileStoreLookupFunction(

public void open(FunctionContext context) throws Exception {
this.functionContext = context;
String tmpDirectory = getTmpDirectory(context);
this.tmpDirectory = getTmpDirectory(context);
open(tmpDirectory);
}

Expand Down Expand Up @@ -236,6 +241,16 @@ 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)
&& lookupTable instanceof FullCacheLookupTable,
table.name(),
this.tmpDirectory,
partitionLoader.partitions());
}
}

if (cacheRowFilter != null) {
Expand Down Expand Up @@ -271,13 +286,16 @@ public Collection<RowData> lookup(RowData keyRow) {
if (partitionLoader == null) {
return lookupInternal(key);
}

if (partitionLoader.partitions().isEmpty()) {
List<BinaryRow> partitions =
partitionRefresher != null
? partitionRefresher.currentPartitions()
: partitionLoader.partitions();
if (partitions.isEmpty()) {
return Collections.emptyList();
}

List<RowData> rows = new ArrayList<>();
for (BinaryRow partition : partitionLoader.partitions()) {
for (BinaryRow partition : partitions) {
rows.addAll(lookupInternal(JoinedRow.join(key, partition)));
}
return rows;
Expand Down Expand Up @@ -324,7 +342,18 @@ 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 newLookupTable =
partitionRefresher.getNewLookupTable(partitionLoader.partitions());
if (newLookupTable != null) {
lookupTable.close();
lookupTable = newLookupTable;
path = partitionRefresher.path();
}
}

// 3. refresh dynamic partition
if (partitionLoader != null) {
boolean partitionChanged = partitionLoader.checkRefresh();
List<BinaryRow> partitions = partitionLoader.partitions();
Expand All @@ -334,18 +363,17 @@ void tryRefresh() throws Exception {
}

if (partitionChanged) {
// reopen with latest partition
lookupTable.specifyPartitions(
partitionLoader.partitions(), partitionLoader.createSpecificPartFilter());
lookupTable.close();
lookupTable.open();
partitionRefresher.startRefresh(
partitions,
partitionLoader.createSpecificPartFilter(),
lookupTable,
cacheRowFilter);
nextRefreshTime = System.currentTimeMillis() + refreshInterval.toMillis();
// no need to refresh the lookup table because it is reopened
return;
}
}

// 3. refresh lookup table
// 4. refresh lookup table
if (shouldRefreshLookupTable()) {
// Check if we should do full load (close and reopen table) instead of incremental
// refresh
Expand Down Expand Up @@ -415,6 +443,10 @@ long nextBlacklistCheckTime() {

@Override
public void close() throws IOException {
if (partitionRefresher != null) {
partitionRefresher.close();
}

if (lookupTable != null) {
lookupTable.close();
lookupTable = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -414,5 +414,16 @@ public Context copy(int[] newProjection) {
joinKey,
requiredCachedBucketIds);
}

public Context copy(File newTempPath) {
return new Context(
table.wrapped(),
projection,
tablePredicate,
projectedPredicate,
newTempPath,
joinKey,
requiredCachedBucketIds);
}
}
}
Loading
Loading