Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[flink] Support lookup join of partial-update and no changelog dim table #4052

Closed
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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.CoreOptions;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.io.SplitsParallelReadUtil;
import org.apache.paimon.mergetree.compact.ConcatRecordReader;
import org.apache.paimon.options.ConfigOption;
import org.apache.paimon.options.Options;
import org.apache.paimon.predicate.Predicate;
import org.apache.paimon.reader.ReaderSupplier;
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.FunctionWithIOException;
import org.apache.paimon.utils.TypeUtils;

import org.apache.paimon.shade.guava30.com.google.common.primitives.Ints;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.IntUnaryOperator;
import java.util.stream.IntStream;

import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_BOOTSTRAP_PARALLELISM;
import static org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping;

/** Reader to load data into {@link LookupTable}. */
abstract class AbstractLookupReader {

protected final Table table;
protected final int[] projection;
protected final ReadBuilder readBuilder;
@Nullable protected final Predicate projectedPredicate;

protected static final List<ConfigOption<?>> TIME_TRAVEL_OPTIONS =
Arrays.asList(
CoreOptions.SCAN_TIMESTAMP_MILLIS,
CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS,
CoreOptions.SCAN_SNAPSHOT_ID,
CoreOptions.SCAN_TAG_NAME,
CoreOptions.SCAN_VERSION);

protected AbstractLookupReader(
Table table,
int[] projection,
@Nullable Predicate predicate,
Set<Integer> requireCachedBucketIds) {
this.table = unsetTimeTravelOptions(table);
this.projection = projection;
this.readBuilder =
this.table
.newReadBuilder()
.withProjection(projection)
.withFilter(predicate)
.withBucketFilter(
requireCachedBucketIds == null
? null
: requireCachedBucketIds::contains);
if (predicate != null) {
List<String> fieldNames = table.rowType().getFieldNames();
List<String> primaryKeys = table.primaryKeys();

// for pk table: only filter by pk, the stream is upsert instead of changelog
// for non-pk table: filter all
IntUnaryOperator operator =
i -> {
int index = Ints.indexOf(projection, i);
boolean safeFilter =
primaryKeys.isEmpty() || primaryKeys.contains(fieldNames.get(i));
return safeFilter ? index : -1;
};

int[] fieldIdxToProjectionIdx =
IntStream.range(0, table.rowType().getFieldCount()).map(operator).toArray();

this.projectedPredicate =
transformFieldMapping(predicate, fieldIdxToProjectionIdx).orElse(null);
} else {
this.projectedPredicate = null;
}
}

protected Table unsetTimeTravelOptions(Table origin) {
FileStoreTable fileStoreTable = (FileStoreTable) origin;
Map<String, String> newOptions = new HashMap<>(fileStoreTable.options());
TIME_TRAVEL_OPTIONS.stream().map(ConfigOption::key).forEach(newOptions::remove);

CoreOptions.StartupMode startupMode = CoreOptions.fromMap(newOptions).startupMode();
if (startupMode != CoreOptions.StartupMode.COMPACTED_FULL) {
startupMode = CoreOptions.StartupMode.LATEST_FULL;
}
newOptions.put(CoreOptions.SCAN_MODE.key(), startupMode.toString());

TableSchema newSchema = fileStoreTable.schema().copy(newOptions);
return fileStoreTable.copy(newSchema);
}

protected abstract List<Split> nextBatchSplits();

protected abstract Long nextSnapshotId();

public RecordReader<InternalRow> nextBatch(boolean useParallelism) throws Exception {
List<Split> splits = nextBatchSplits();
return nextBatch(useParallelism, splits);
}

public RecordReader<InternalRow> nextBatch(boolean useParallelism, List<Split> splits)
throws Exception {

CoreOptions options = CoreOptions.fromMap(table.options());
FunctionWithIOException<Split, RecordReader<InternalRow>> readerSupplier =
split -> readBuilder.newRead().createReader(split);

RowType readType = TypeUtils.project(table.rowType(), projection);

RecordReader<InternalRow> reader;
if (useParallelism) {
reader =
SplitsParallelReadUtil.parallelExecute(
readType,
readerSupplier,
splits,
options.pageSize(),
new Options(table.options()).get(LOOKUP_BOOTSTRAP_PARALLELISM));
} else {
List<ReaderSupplier<InternalRow>> readers = new ArrayList<>();
for (Split split : splits) {
readers.add(() -> readerSupplier.apply(split));
}
reader = ConcatRecordReader.create(readers);
}

if (projectedPredicate != null) {
reader = reader.filter(projectedPredicate::test);
}
return reader;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.paimon.flink.FlinkConnectorOptions.LookupCacheMode;
import org.apache.paimon.flink.FlinkRowData;
import org.apache.paimon.flink.FlinkRowWrapper;
import org.apache.paimon.flink.utils.TableScanUtils;
import org.apache.paimon.options.Options;
import org.apache.paimon.predicate.Predicate;
import org.apache.paimon.table.FileStoreTable;
Expand Down Expand Up @@ -98,8 +97,6 @@ public class FileStoreLookupFunction implements Serializable, Closeable {

public FileStoreLookupFunction(
Table table, int[] projection, int[] joinKeyIndex, @Nullable Predicate predicate) {
TableScanUtils.streamingReadingValidate(table);

this.table = table;
this.partitionLoader = DynamicPartitionLoader.of(table);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,8 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;

import static org.apache.paimon.CoreOptions.ChangelogProducer.NONE;
import static org.apache.paimon.CoreOptions.MergeEngine.PARTIAL_UPDATE;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC;
import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC_PENDING_SNAPSHOT_COUNT;

Expand All @@ -83,7 +85,7 @@ public abstract class FullCacheLookupTable implements LookupTable {
private final int maxPendingSnapshotCount;
private final FileStoreTable table;
private Future<?> refreshFuture;
private LookupStreamingReader reader;
private AbstractLookupReader reader;
private Predicate specificPartition;

public FullCacheLookupTable(Context context) {
Expand Down Expand Up @@ -149,12 +151,20 @@ protected void openStateFactory() throws Exception {
protected void bootstrap() throws Exception {
Predicate scanPredicate =
PredicateBuilder.andNullable(context.tablePredicate, specificPartition);

CoreOptions options = CoreOptions.fromMap(table.options());
this.reader =
new LookupStreamingReader(
context.table,
context.projection,
scanPredicate,
context.requiredCachedBucketIds);
(options.mergeEngine() == PARTIAL_UPDATE && options.changelogProducer() == NONE)
? new LookupBatchReader(
context.table,
context.projection,
scanPredicate,
context.requiredCachedBucketIds)
: new LookupStreamingReader(
context.table,
context.projection,
scanPredicate,
context.requiredCachedBucketIds);
BinaryExternalSortBuffer bulkLoadSorter =
RocksDBState.createBulkLoadSorter(
IOManager.create(context.tempPath.toString()), context.table.coreOptions());
Expand Down Expand Up @@ -236,14 +246,24 @@ public void refresh() throws Exception {
}

private void doRefresh() throws Exception {
while (true) {
if (reader instanceof LookupBatchReader) {
try (RecordReaderIterator<InternalRow> batch =
new RecordReaderIterator<>(reader.nextBatch(false))) {
if (!batch.hasNext()) {
return;
}
refresh(batch);
}
} else {
while (true) {
try (RecordReaderIterator<InternalRow> batch =
new RecordReaderIterator<>(reader.nextBatch(false))) {
if (!batch.hasNext()) {
return;
}
refresh(batch);
}
}
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.predicate.Predicate;
import org.apache.paimon.table.Table;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.table.source.snapshot.SnapshotReader;

import javax.annotation.Nullable;

import java.util.List;
import java.util.Set;

/** A batch reader to load data into {@link LookupTable}. */
public class LookupBatchReader extends AbstractLookupReader {
private TableScan scan;
private Long nextSnapshotId;

public LookupBatchReader(
Table table,
int[] projection,
@Nullable Predicate predicate,
Set<Integer> requireCachedBucketIds) {
super(table, projection, predicate, requireCachedBucketIds);
}

@Override
protected List<Split> nextBatchSplits() {
SnapshotReader.Plan plan = (SnapshotReader.Plan) readBuilder.newScan().plan();
nextSnapshotId = plan.snapshotId() == null ? 1 : plan.snapshotId() + 1;
return readBuilder.newScan().plan().splits();
}

@Nullable
public Long nextSnapshotId() {
return nextSnapshotId;
}
}
Loading
Loading