|
34 | 34 | import org.apache.flink.connector.jdbc.core.datastream.source.enumerator.JdbcSourceEnumerator;
|
35 | 35 | import org.apache.flink.connector.jdbc.core.datastream.source.enumerator.JdbcSourceEnumeratorState;
|
36 | 36 | import org.apache.flink.connector.jdbc.core.datastream.source.enumerator.JdbcSqlSplitEnumeratorBase;
|
| 37 | +import org.apache.flink.connector.jdbc.core.datastream.source.enumerator.SqlTemplateSplitEnumerator; |
37 | 38 | import org.apache.flink.connector.jdbc.core.datastream.source.reader.JdbcSourceReader;
|
38 | 39 | import org.apache.flink.connector.jdbc.core.datastream.source.reader.JdbcSourceSplitReader;
|
39 | 40 | import org.apache.flink.connector.jdbc.core.datastream.source.reader.extractor.ResultExtractor;
|
40 | 41 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit;
|
41 | 42 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplitSerializer;
|
42 | 43 | import org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
|
| 44 | +import org.apache.flink.connector.jdbc.lineage.DefaultTypeDatasetFacet; |
| 45 | +import org.apache.flink.connector.jdbc.lineage.LineageUtils; |
43 | 46 | import org.apache.flink.connector.jdbc.utils.ContinuousUnBoundingSettings;
|
44 | 47 | import org.apache.flink.core.io.SimpleVersionedSerializer;
|
| 48 | +import org.apache.flink.streaming.api.lineage.LineageDataset; |
| 49 | +import org.apache.flink.streaming.api.lineage.LineageVertex; |
| 50 | +import org.apache.flink.streaming.api.lineage.LineageVertexProvider; |
45 | 51 | import org.apache.flink.util.Preconditions;
|
46 | 52 |
|
47 | 53 | import javax.annotation.Nullable;
|
48 | 54 |
|
49 | 55 | import java.io.Serializable;
|
50 | 56 | import java.util.ArrayList;
|
| 57 | +import java.util.Arrays; |
| 58 | +import java.util.Collections; |
51 | 59 | import java.util.Objects;
|
| 60 | +import java.util.Optional; |
52 | 61 |
|
53 | 62 | /** JDBC source. */
|
54 | 63 | @PublicEvolving
|
55 | 64 | public class JdbcSource<OUT>
|
56 |
| - implements Source<OUT, JdbcSourceSplit, JdbcSourceEnumeratorState>, |
| 65 | + implements LineageVertexProvider, |
| 66 | + Source<OUT, JdbcSourceSplit, JdbcSourceEnumeratorState>, |
57 | 67 | ResultTypeQueryable<OUT> {
|
58 | 68 |
|
59 | 69 | private final Boundedness boundedness;
|
@@ -195,4 +205,18 @@ public boolean equals(Object o) {
|
195 | 205 | && deliveryGuarantee == that.deliveryGuarantee
|
196 | 206 | && Objects.equals(continuousUnBoundingSettings, that.continuousUnBoundingSettings);
|
197 | 207 | }
|
| 208 | + |
| 209 | + @Override |
| 210 | + public LineageVertex getLineageVertex() { |
| 211 | + DefaultTypeDatasetFacet defaultTypeDatasetFacet = |
| 212 | + new DefaultTypeDatasetFacet(getTypeInformation()); |
| 213 | + SqlTemplateSplitEnumerator enumerator = |
| 214 | + (SqlTemplateSplitEnumerator) sqlSplitEnumeratorProvider.create(); |
| 215 | + Optional<String> nameOpt = LineageUtils.nameOf(enumerator.getSqlTemplate()); |
| 216 | + String namespace = LineageUtils.namespaceOf(connectionProvider); |
| 217 | + LineageDataset dataset = |
| 218 | + LineageUtils.datasetOf( |
| 219 | + nameOpt.orElse(""), namespace, Arrays.asList(defaultTypeDatasetFacet)); |
| 220 | + return LineageUtils.sourceLineageVertexOf(boundedness, Collections.singleton(dataset)); |
| 221 | + } |
198 | 222 | }
|
0 commit comments