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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.paimon.flink.lookup;

import org.apache.paimon.CoreOptions;
import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.InternalRow;
Expand Down Expand Up @@ -79,7 +80,7 @@ public abstract class FullCacheLookupTable implements LookupTable {
protected final int appendUdsFieldNumber;

protected RocksDBStateFactory stateFactory;
@Nullable private final ExecutorService refreshExecutor;
@Nullable private ExecutorService refreshExecutor;
private final AtomicReference<Exception> cachedException;
private final int maxPendingSnapshotCount;
private final FileStoreTable table;
Expand Down Expand Up @@ -127,14 +128,6 @@ public FullCacheLookupTable(Context context) {
Options options = Options.fromMap(context.table.options());
this.projectedType = projectedType;
this.refreshAsync = options.get(LOOKUP_REFRESH_ASYNC);
this.refreshExecutor =
this.refreshAsync
? Executors.newSingleThreadExecutor(
new ExecutorThreadFactory(
String.format(
"%s-lookup-refresh",
Thread.currentThread().getName())))
: null;
this.cachedException = new AtomicReference<>();
this.maxPendingSnapshotCount = options.get(LOOKUP_REFRESH_ASYNC_PENDING_SNAPSHOT_COUNT);
}
Expand All @@ -149,12 +142,20 @@ public void specifyCacheRowFilter(Filter<InternalRow> filter) {
this.cacheRowFilter = filter;
}

protected void openStateFactory() throws Exception {
protected void init() throws Exception {
this.stateFactory =
new RocksDBStateFactory(
context.tempPath.toString(),
context.table.coreOptions().toConfiguration(),
null);
this.refreshExecutor =
this.refreshAsync
? Executors.newSingleThreadExecutor(
new ExecutorThreadFactory(
String.format(
"%s-lookup-refresh",
Thread.currentThread().getName())))
: null;
}

protected void bootstrap() throws Exception {
Expand Down Expand Up @@ -322,6 +323,11 @@ public void close() throws IOException {
}
}

@VisibleForTesting
public Future<?> getRefreshFuture() {
return refreshFuture;
}

/** Bulk loader for the table. */
public interface TableBulkLoader {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public NoPrimaryKeyLookupTable(Context context, long lruCacheSize) {

@Override
public void open() throws Exception {
openStateFactory();
init();
this.state =
stateFactory.listState(
"join-key-index",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ public PrimaryKeyLookupTable(Context context, long lruCacheSize, List<String> jo

@Override
public void open() throws Exception {
openStateFactory();
init();
createTableState();
bootstrap();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public SecondaryIndexLookupTable(Context context, long lruCacheSize) {

@Override
public void open() throws Exception {
openStateFactory();
init();
createTableState();
this.indexState =
stateFactory.setState(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -600,6 +600,41 @@ public void testPkTableWithCacheRowFilter() throws Exception {
assertThat(res).isEmpty();
}

@Test
public void testRefreshExecutorRebuildAfterReopen() throws Exception {
Options options = new Options();
options.set(FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC, true);
FileStoreTable storeTable = createTable(singletonList("f0"), options);
writeWithBucketAssigner(
storeTable, row -> 0, GenericRow.of(1, 11, 111), GenericRow.of(2, 22, 222));

FullCacheLookupTable.Context context =
new FullCacheLookupTable.Context(
storeTable,
new int[] {0, 1, 2},
null,
null,
tempDir.toFile(),
singletonList("f0"),
null);
table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10);
assertThat(table).isInstanceOf(PrimaryKeyLookupTable.class);
table.open();
// reopen
table.close();
table.open();
List<InternalRow> res = table.get(GenericRow.of(1));
assertThat(res).hasSize(1);
assertRow(res.get(0), 1, 11, 111);
writeWithBucketAssigner(storeTable, row -> 0, GenericRow.of(1, 22, 222));
table.refresh();
assertThat(table.getRefreshFuture()).isNotNull();
table.getRefreshFuture().get();
res = table.get(GenericRow.of(1));
assertThat(res).hasSize(1);
assertRow(res.get(0), 1, 22, 222);
}

@Test
public void testNoPkTableWithCacheRowFilter() throws Exception {
FileStoreTable storeTable = createTable(emptyList(), new Options());
Expand Down
Loading