Skip to content
Open
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 @@ -18,10 +18,14 @@

package org.apache.paimon.flink;

import org.apache.paimon.flink.lineage.DataStreamProviderFactory;
import org.apache.paimon.table.Table;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.connector.ProviderContext;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
import org.apache.flink.table.data.RowData;

import java.util.function.Function;
Expand All @@ -38,6 +42,19 @@ public PaimonDataStreamScanProvider(
this.producer = producer;
}

/**
* Creates a {@link ScanRuntimeProvider} that may be enriched with lineage metadata when running
* on a Flink version that supports it.
*/
public static ScanRuntimeProvider createProvider(
boolean isBounded,
Function<StreamExecutionEnvironment, DataStream<RowData>> producer,
String name,
Table table) {
return DataStreamProviderFactory.getScanProvider(
new PaimonDataStreamScanProvider(isBounded, producer), name, table);
}

@Override
public DataStream<RowData> produceDataStream(
ProviderContext context, StreamExecutionEnvironment env) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,14 @@

package org.apache.paimon.flink;

import org.apache.paimon.flink.lineage.DataStreamProviderFactory;
import org.apache.paimon.table.Table;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.table.connector.ProviderContext;
import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
import org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
import org.apache.flink.table.data.RowData;

import java.util.function.Function;
Expand All @@ -35,6 +39,16 @@ public PaimonDataStreamSinkProvider(Function<DataStream<RowData>, DataStreamSink
this.producer = producer;
}

/**
* Creates a {@link SinkRuntimeProvider} that may be enriched with lineage metadata when running
* on a Flink version that supports it.
*/
public static SinkRuntimeProvider createProvider(
Function<DataStream<RowData>, DataStreamSink<?>> producer, String name, Table table) {
return DataStreamProviderFactory.getSinkProvider(
new PaimonDataStreamSinkProvider(producer), name, table);
}

@Override
public DataStreamSink<?> consumeDataStream(
ProviderContext providerContext, DataStream<RowData> dataStream) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,10 +57,12 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {

@Override
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
return new PaimonDataStreamSinkProvider(
return PaimonDataStreamSinkProvider.createProvider(
(dataStream) ->
new FlinkFormatTableDataStreamSink(table, overwrite, staticPartitions)
.sinkFrom(dataStream));
.sinkFrom(dataStream),
tableIdentifier.asSummaryString(),
table);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,18 +104,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
throw new UnsupportedOperationException(
"Paimon doesn't support streaming INSERT OVERWRITE.");
}
String name = tableIdentifier.asSummaryString();

if (table instanceof FormatTable) {
FormatTable formatTable = (FormatTable) table;
return new PaimonDataStreamSinkProvider(
return PaimonDataStreamSinkProvider.createProvider(
(dataStream) ->
new FlinkFormatTableDataStreamSink(
formatTable, overwrite, staticPartitions)
.sinkFrom(dataStream));
.sinkFrom(dataStream),
name,
table);
}

Options conf = Options.fromMap(table.options());
// Do not sink to log store when overwrite mode
return new PaimonDataStreamSinkProvider(
return PaimonDataStreamSinkProvider.createProvider(
(dataStream) -> {
FlinkSinkBuilder builder = createSinkBuilder();
builder.forRowData(
Expand All @@ -134,7 +138,9 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
}
conf.getOptional(SINK_PARALLELISM).ifPresent(builder::parallelism);
return builder.build();
});
},
name,
table);
}

protected FlinkSinkBuilder createSinkBuilder() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,14 +195,15 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
.limit(limit)
.watermarkStrategy(watermarkStrategy)
.dynamicPartitionFilteringFields(dynamicPartitionFilteringFields());

return new PaimonDataStreamScanProvider(
return PaimonDataStreamScanProvider.createProvider(
!unbounded,
env ->
sourceBuilder
.sourceParallelism(inferSourceParallelism(env))
.env(env)
.build());
.build(),
tableIdentifier.asSummaryString(),
table);
}

private ScanRuntimeProvider createCountStarScan() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
CoreOptions.BLOB_AS_DESCRIPTOR.key(),
"false")));
}
return new PaimonDataStreamScanProvider(
return PaimonDataStreamScanProvider.createProvider(
source.getBoundedness() == Boundedness.BOUNDED,
env -> {
Integer parallelism = inferSourceParallelism(env);
Expand All @@ -130,7 +130,9 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
dataStreamSource.setParallelism(parallelism);
}
return dataStreamSource;
});
},
tableIdentifier.asSummaryString(),
table);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.paimon.data.GenericRow;
import org.apache.paimon.flink.FlinkConnectorOptions;
import org.apache.paimon.flink.PaimonDataStreamScanProvider;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.local.LocalFileIO;
Expand All @@ -43,6 +42,7 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
Expand Down Expand Up @@ -75,7 +75,7 @@ void testInferScanParallelism() throws Exception {
DataTableSource tableSource =
new DataTableSource(
ObjectIdentifier.of("cat", "db", "table"), fileStoreTable, true, null);
PaimonDataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);
DataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);
StreamExecutionEnvironment sEnv1 = StreamExecutionEnvironment.createLocalEnvironment();
sEnv1.setParallelism(-1);
DataStream<RowData> sourceStream1 =
Expand Down Expand Up @@ -105,7 +105,7 @@ public void testInferStreamParallelism() throws Exception {
DataTableSource tableSource =
new DataTableSource(
ObjectIdentifier.of("cat", "db", "table"), fileStoreTable, true, null);
PaimonDataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);
DataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);

StreamExecutionEnvironment sEnv1 = StreamExecutionEnvironment.createLocalEnvironment();
DataStream<RowData> sourceStream1 =
Expand All @@ -123,7 +123,7 @@ public void testSystemTableParallelism() throws Exception {

SystemTableSource tableSource =
new SystemTableSource(ro, false, ObjectIdentifier.of("cat", "db", "table"));
PaimonDataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);
DataStreamScanProvider runtimeProvider = runtimeProvider(tableSource);

Configuration configuration = new Configuration();
configuration.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH);
Expand All @@ -133,8 +133,8 @@ public void testSystemTableParallelism() throws Exception {
assertThat(sourceStream1.getParallelism()).isEqualTo(3);
}

private PaimonDataStreamScanProvider runtimeProvider(FlinkTableSource tableSource) {
return (PaimonDataStreamScanProvider)
private DataStreamScanProvider runtimeProvider(FlinkTableSource tableSource) {
return (DataStreamScanProvider)
tableSource.getScanRuntimeProvider(
new ScanTableSource.ScanContext() {
@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.lineage;

import org.apache.paimon.table.Table;

import org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;

/** Stub factory for Flink 1.x. Returns providers unchanged since lineage is not supported. */
public class DataStreamProviderFactory {

/** Returns the provider unchanged. Flink 1.x does not support lineage. */
public static ScanRuntimeProvider getScanProvider(
ScanRuntimeProvider provider, String name, Table table) {
return provider;
}

/** Returns the provider unchanged. Flink 1.x does not support lineage. */
public static SinkRuntimeProvider getSinkProvider(
SinkRuntimeProvider provider, String name, Table table) {
return provider;
}
}
6 changes: 6 additions & 0 deletions paimon-flink/paimon-flink2-common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,12 @@ under the License.
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.lineage;

import org.apache.paimon.table.Table;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.lineage.LineageVertex;
import org.apache.flink.streaming.api.lineage.LineageVertexProvider;
import org.apache.flink.table.connector.ProviderContext;
import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
import org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
import org.apache.flink.table.data.RowData;

/**
* Factory that wraps {@link DataStreamScanProvider} and {@link DataStreamSinkProvider} with {@link
* LineageVertexProvider} support for Flink 2.0+.
*/
public class DataStreamProviderFactory {

/**
* Returns a {@link ScanRuntimeProvider} that also implements {@link LineageVertexProvider} so
* Flink's lineage graph discovers the Paimon source table.
*/
public static ScanRuntimeProvider getScanProvider(
ScanRuntimeProvider provider, String name, Table table) {
return new LineageAwarePaimonDataStreamScanProvider(
(DataStreamScanProvider) provider, name, table);
}

/**
* Returns a {@link SinkRuntimeProvider} that also implements {@link LineageVertexProvider} so
* Flink's lineage graph discovers the Paimon sink table.
*/
public static SinkRuntimeProvider getSinkProvider(
SinkRuntimeProvider provider, String name, Table table) {
return new LineageAwarePaimonDataStreamSinkProvider(
(DataStreamSinkProvider) provider, name, table);
}

private static class LineageAwarePaimonDataStreamScanProvider
implements DataStreamScanProvider, LineageVertexProvider {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Compared with delegate pattern, would it be better to directly have PaimonDataStreamScanProvider implement LineageVertexProvider?

Same for LineageAwarePaimonDataStreamSinkProvider.

Copy link
Contributor Author

@jsingh-yelp jsingh-yelp Mar 9, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review @yunfengzhou-hub. The reason for the delegate pattern is backward compatibility. PaimonDataStreamScanProvider lives in paimon-flink-common, which is built against multiple Flink versions, including ones older than 1.20. Since LineageVertexProvider only exists in Flink 1.20+, having PaimonDataStreamScanProvider directly implement it would cause a ClassNotFoundException at build/load time for older Flink versions.

By using the delegate pattern with a version-specific DataStreamProviderFactory, the lineage-aware wrapper is only created in the paimon-flink2-common module (which targets Flink 2.x), and the older Flink modules never reference LineageVertexProvider at all.


private final DataStreamScanProvider delegate;
private final String name;
private final Table table;

LineageAwarePaimonDataStreamScanProvider(
DataStreamScanProvider delegate, String name, Table table) {
this.delegate = delegate;
this.name = name;
this.table = table;
}

@Override
public DataStream<RowData> produceDataStream(
ProviderContext context, StreamExecutionEnvironment env) {
return delegate.produceDataStream(context, env);
}

@Override
public boolean isBounded() {
return delegate.isBounded();
}

@Override
public LineageVertex getLineageVertex() {
return LineageUtils.sourceLineageVertex(name, delegate.isBounded(), table);
}
}

private static class LineageAwarePaimonDataStreamSinkProvider
implements DataStreamSinkProvider, LineageVertexProvider {

private final DataStreamSinkProvider delegate;
private final String name;
private final Table table;

LineageAwarePaimonDataStreamSinkProvider(
DataStreamSinkProvider delegate, String name, Table table) {
this.delegate = delegate;
this.name = name;
this.table = table;
}

@Override
public DataStreamSink<?> consumeDataStream(
ProviderContext providerContext, DataStream<RowData> dataStream) {
return delegate.consumeDataStream(providerContext, dataStream);
}

@Override
public LineageVertex getLineageVertex() {
return LineageUtils.sinkLineageVertex(name, table);
}
}
}
Loading