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 @@ -33,7 +33,10 @@
import org.apache.flink.api.connector.source.SourceReaderContext;
import org.apache.flink.api.connector.source.SplitEnumerator;
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Preconditions;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
Expand All @@ -50,12 +53,12 @@
import org.apache.iceberg.flink.source.reader.IcebergSourceReader;
import org.apache.iceberg.flink.source.reader.ReaderFunction;
import org.apache.iceberg.flink.source.reader.ReaderMetricsContext;
import org.apache.iceberg.flink.source.reader.RowDataReaderFunction;
import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


@Experimental
public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEnumeratorState> {
private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class);
Expand Down Expand Up @@ -150,12 +153,17 @@ public static <T> Builder<T> builder() {
return new Builder<>();
}

public static Builder<RowData> forRowData() {
return new Builder<>();
}

public static class Builder<T> {

// required
private TableLoader tableLoader;
private SplitAssignerFactory splitAssignerFactory;
private ReaderFunction<T> readerFunction;
private ReadableConfig flinkConfig = new Configuration();

// optional
private final ScanContext.Builder contextBuilder = ScanContext.builder();
Expand All @@ -178,6 +186,11 @@ public Builder<T> readerFunction(ReaderFunction<T> newReaderFunction) {
return this;
}

public Builder<T> flinkConfig(ReadableConfig config) {
this.flinkConfig = config;
return this;
}

public Builder caseSensitive(boolean newCaseSensitive) {
this.contextBuilder.caseSensitive(newCaseSensitive);
return this;
Expand Down Expand Up @@ -274,12 +287,21 @@ public Builder properties(Map<String, String> properties) {
}

public IcebergSource<T> build() {
ScanContext context = contextBuilder.build();
if (readerFunction == null) {
try (TableLoader loader = tableLoader) {
loader.open();
Table table = tableLoader.loadTable();
RowDataReaderFunction rowDataReaderFunction = new RowDataReaderFunction(flinkConfig, table.schema(),
context.project(), context.nameMapping(), context.caseSensitive(), table.io(), table.encryption());
this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction;
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}

checkRequired();
return new IcebergSource<T>(
tableLoader,
contextBuilder.build(),
readerFunction,
splitAssignerFactory);
return new IcebergSource<T>(tableLoader, context, readerFunction, splitAssignerFactory);
}

private void checkRequired() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@

import java.io.IOException;
import java.util.NoSuchElementException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.SourceReaderOptions;
import org.apache.flink.connector.file.src.util.Pool;
Expand All @@ -40,9 +40,9 @@ class ArrayPoolDataIteratorBatcher<T> implements DataIteratorBatcher<T> {

private transient Pool<T[]> pool;

ArrayPoolDataIteratorBatcher(Configuration config, RecordFactory<T> recordFactory) {
this.batchSize = config.getInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT);
this.handoverQueueSize = config.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY);
ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory<T> recordFactory) {
this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT);
this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY);
this.recordFactory = recordFactory;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

package org.apache.iceberg.flink.source.reader;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.data.RowData;
import org.apache.iceberg.Schema;
import org.apache.iceberg.encryption.EncryptionManager;
Expand All @@ -39,7 +39,7 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction<RowData> {
private final EncryptionManager encryption;

public RowDataReaderFunction(
Configuration config, Schema tableSchema, Schema projectedSchema,
ReadableConfig config, Schema tableSchema, Schema projectedSchema,
String nameMapping, boolean caseSensitive, FileIO io, EncryptionManager encryption) {
super(new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(
FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema)))));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
import org.apache.iceberg.flink.TestFixtures;
import org.apache.iceberg.flink.data.RowDataToRowMapper;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.flink.source.reader.RowDataReaderFunction;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
Expand Down Expand Up @@ -95,11 +94,10 @@ private List<Row> run(Schema projectedSchema, List<Expression> filters,
table = tableLoader.loadTable();
}

IcebergSource.Builder<RowData> sourceBuilder = IcebergSource.<RowData>builder()
IcebergSource.Builder<RowData> sourceBuilder = IcebergSource.forRowData()
.tableLoader(tableLoader())
.assignerFactory(new SimpleSplitAssignerFactory())
.readerFunction(new RowDataReaderFunction(config, table.schema(), projectedSchema,
null, false, table.io(), table.encryption()));
.flinkConfig(config);
if (projectedSchema != null) {
sourceBuilder.project(projectedSchema);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.client.JobStatusMessage;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
Expand All @@ -38,7 +37,6 @@
import org.apache.flink.types.Row;
import org.apache.flink.util.CloseableIterator;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Table;
import org.apache.iceberg.data.GenericAppenderHelper;
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
Expand All @@ -49,7 +47,6 @@
import org.apache.iceberg.flink.TestHelpers;
import org.apache.iceberg.flink.data.RowDataToRowMapper;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.flink.source.reader.RowDataReaderFunction;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.junit.ClassRule;
import org.junit.Rule;
Expand Down Expand Up @@ -301,17 +298,13 @@ public void testSpecificSnapshotTimestamp() throws Exception {
}

private DataStream<Row> createStream(ScanContext scanContext) throws Exception {
Table table = tableResource.table();
Configuration config = new Configuration();
// start the source and collect output
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
DataStream<Row> stream = env.fromSource(
IcebergSource.<RowData>builder()
IcebergSource.forRowData()
.tableLoader(tableResource.tableLoader())
.assignerFactory(new SimpleSplitAssignerFactory())
.readerFunction(new RowDataReaderFunction(config, table.schema(), null,
null, false, table.io(), table.encryption()))
.streaming(scanContext.isStreaming())
.streamingStartingStrategy(scanContext.streamingStartingStrategy())
.startSnapshotTimestamp(scanContext.startSnapshotTimestamp())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
import org.apache.iceberg.flink.TestFixtures;
import org.apache.iceberg.flink.sink.FlinkSink;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.flink.source.reader.RowDataReaderFunction;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.junit.ClassRule;
import org.junit.Rule;
Expand Down Expand Up @@ -84,14 +83,12 @@ public class TestIcebergSourceFailover {
TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema());

protected IcebergSource.Builder<RowData> sourceBuilder() {
Table sourceTable = sourceTableResource.table();
Configuration config = new Configuration();
config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128);
return IcebergSource.<RowData>builder()
return IcebergSource.forRowData()
.tableLoader(sourceTableResource.tableLoader())
.assignerFactory(new SimpleSplitAssignerFactory())
.readerFunction(new RowDataReaderFunction(config, sourceTable.schema(), null,
null, false, sourceTable.io(), sourceTable.encryption()));
.flinkConfig(config);
}

protected Schema schema() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.util.Map;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
Expand All @@ -43,7 +42,6 @@
import org.apache.iceberg.flink.RowDataWrapper;
import org.apache.iceberg.flink.TableLoader;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.flink.source.reader.RowDataReaderFunction;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.StructLikeSet;
Expand Down Expand Up @@ -82,16 +80,12 @@ protected StructLikeSet rowSet(String tableName, Table testTable, String... colu
TableLoader hiveTableLoader = TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName));
hiveTableLoader.open();
try (TableLoader tableLoader = hiveTableLoader) {
Configuration config = new Configuration();
Table table = tableLoader.loadTable();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
DataStream<RowData> stream = env.fromSource(
IcebergSource.<RowData>builder()
.tableLoader(tableLoader)
.assignerFactory(new SimpleSplitAssignerFactory())
.readerFunction(new RowDataReaderFunction(config, table.schema(), projected,
null, false, table.io(), table.encryption()))
.project(projected)
.build(),
WatermarkStrategy.noWatermarks(),
Expand Down